From 7ddadd25a3a0217f55ad8dcb8adfbf8e643b3d0c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 26 Jul 2022 09:08:55 +0000 Subject: [PATCH 01/43] Initial implementation of KeeperMap --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Storages/KVStorageUtils.cpp | 175 ++++++++++ src/Storages/KVStorageUtils.h | 47 +++ .../RocksDB/StorageEmbeddedRocksDB.cpp | 186 +--------- src/Storages/StorageKeeperMap.cpp | 329 ++++++++++++++++++ src/Storages/StorageKeeperMap.h | 52 +++ src/Storages/registerStorages.cpp | 3 + 7 files changed, 608 insertions(+), 186 deletions(-) create mode 100644 src/Storages/KVStorageUtils.cpp create mode 100644 src/Storages/KVStorageUtils.h create mode 100644 src/Storages/StorageKeeperMap.cpp create mode 100644 src/Storages/StorageKeeperMap.h diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 7a00bbf524c..ea11a1b23ac 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -720,7 +720,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti } /// We can have queries like "CREATE TABLE ENGINE=" if /// supports schema inference (will determine table structure in it's constructor). - else if (!StorageFactory::instance().checkIfStorageSupportsSchemaInterface(create.storage->engine->name)) // NOLINT + else if (!StorageFactory::instance().checkIfStorageSupportsSchemaInterface(create.storage->engine->name) && create.storage->engine->name != "KeeperMap") // NOLINT throw Exception("Incorrect CREATE query: required list of column descriptions or AS section or SELECT.", ErrorCodes::INCORRECT_QUERY); /// Even if query has list of columns, canonicalize it (unfold Nested columns). diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp new file mode 100644 index 00000000000..8c99d593f40 --- /dev/null +++ b/src/Storages/KVStorageUtils.cpp @@ -0,0 +1,175 @@ +#include + +#include +#include +#include +#include +#include + +#include +#include +#include + +namespace DB +{ + +namespace +{ +// returns keys may be filter by condition +bool traverseASTFilter( + const std::string & primary_key, const DataTypePtr & primary_key_type, const ASTPtr & elem, const PreparedSets & sets, const ContextPtr & context, FieldVectorPtr & res) +{ + const auto * function = elem->as(); + if (!function) + return false; + + if (function->name == "and") + { + // one child has the key filter condition is ok + for (const auto & child : function->arguments->children) + if (traverseASTFilter(primary_key, primary_key_type, child, sets, context, res)) + return true; + return false; + } + else if (function->name == "or") + { + // make sure every child has the key filter condition + for (const auto & child : function->arguments->children) + if (!traverseASTFilter(primary_key, primary_key_type, child, sets, context, res)) + return false; + return true; + } + else if (function->name == "equals" || function->name == "in") + { + const auto & args = function->arguments->as(); + const ASTIdentifier * ident; + std::shared_ptr value; + + if (args.children.size() != 2) + return false; + + if (function->name == "in") + { + ident = args.children.at(0)->as(); + if (!ident) + return false; + + if (ident->name() != primary_key) + return false; + value = args.children.at(1); + + PreparedSetKey set_key; + if ((value->as() || value->as())) + set_key = PreparedSetKey::forSubquery(*value); + else + set_key = PreparedSetKey::forLiteral(*value, {primary_key_type}); + + auto set_it = sets.find(set_key); + if (set_it == sets.end()) + return false; + SetPtr prepared_set = set_it->second; + + if (!prepared_set->hasExplicitSetElements()) + return false; + + prepared_set->checkColumnsNumber(1); + const auto & set_column = *prepared_set->getSetElements()[0]; + for (size_t row = 0; row < set_column.size(); ++row) + res->push_back(set_column[row]); + return true; + } + else + { + if ((ident = args.children.at(0)->as())) + value = args.children.at(1); + else if ((ident = args.children.at(1)->as())) + value = args.children.at(0); + else + return false; + + if (ident->name() != primary_key) + return false; + + const auto node = evaluateConstantExpressionAsLiteral(value, context); + /// function->name == "equals" + if (const auto * literal = node->as()) + { + auto converted_field = convertFieldToType(literal->value, *primary_key_type); + if (!converted_field.isNull()) + res->push_back(converted_field); + return true; + } + } + } + return false; +} +} + +std::pair getFilterKeys( + const String & primary_key, const DataTypePtr & primary_key_type, const SelectQueryInfo & query_info, const ContextPtr & context) +{ + const auto & select = query_info.query->as(); + if (!select.where()) + return {{}, true}; + + FieldVectorPtr res = std::make_shared(); + auto matched_keys = traverseASTFilter(primary_key, primary_key_type, select.where(), query_info.sets, context, res); + return std::make_pair(res, !matched_keys); +} + +std::vector serializeKeysToRawString( + FieldVector::const_iterator & it, + FieldVector::const_iterator end, + DataTypePtr key_column_type, + size_t max_block_size) +{ + size_t num_keys = end - it; + + std::vector result; + result.reserve(num_keys); + + size_t rows_processed = 0; + while (it < end && (max_block_size == 0 || rows_processed < max_block_size)) + { + std::string & serialized_key = result.emplace_back(); + WriteBufferFromString wb(serialized_key); + key_column_type->getDefaultSerialization()->serializeBinary(*it, wb); + wb.finalize(); + + ++it; + ++rows_processed; + } + return result; +} + +std::vector serializeKeysToRawString(const ColumnWithTypeAndName & keys) +{ + if (!keys.column) + return {}; + + size_t num_keys = keys.column->size(); + std::vector result; + result.reserve(num_keys); + + for (size_t i = 0; i < num_keys; ++i) + { + std::string & serialized_key = result.emplace_back(); + WriteBufferFromString wb(serialized_key); + Field field; + keys.column->get(i, field); + /// TODO(@vdimir): use serializeBinaryBulk + keys.type->getDefaultSerialization()->serializeBinary(field, wb); + wb.finalize(); + } + return result; +} + +/// In current implementation rocks db can have key with only one column. +size_t getPrimaryKeyPos(const Block & header, const Names & primary_key) +{ + if (primary_key.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "RocksDB: only one primary key is supported"); + return header.getPositionByName(primary_key[0]); +} + +} diff --git a/src/Storages/KVStorageUtils.h b/src/Storages/KVStorageUtils.h new file mode 100644 index 00000000000..ca6aecd1374 --- /dev/null +++ b/src/Storages/KVStorageUtils.h @@ -0,0 +1,47 @@ +#pragma once + +#include + +#include +#include + +#include + +namespace DB +{ + +using FieldVectorPtr = std::shared_ptr; + +class IDataType; +using DataTypePtr = std::shared_ptr; + +/** Retrieve from the query a condition of the form `key = 'key'`, `key in ('xxx_'), from conjunctions in the WHERE clause. + * TODO support key like search + */ +std::pair getFilterKeys( + const std::string & primary_key, const DataTypePtr & primary_key_type, const SelectQueryInfo & query_info, const ContextPtr & context); + +template +void fillColumns(const K & key, const V & value, size_t key_pos, const Block & header, MutableColumns & columns) +{ + ReadBufferFromString key_buffer(key); + ReadBufferFromString value_buffer(value); + for (size_t i = 0; i < header.columns(); ++i) + { + const auto & serialization = header.getByPosition(i).type->getDefaultSerialization(); + serialization->deserializeBinary(*columns[i], i == key_pos ? key_buffer : value_buffer); + } +} + +std::vector serializeKeysToRawString( + FieldVector::const_iterator & it, + FieldVector::const_iterator end, + DataTypePtr key_column_type, + size_t max_block_size); + +std::vector serializeKeysToRawString(const ColumnWithTypeAndName & keys); + +/// In current implementation key with only column is supported. +size_t getPrimaryKeyPos(const Block & header, const Names & primary_key); + +} diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 2774c52fe7c..dda27c281cb 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -3,30 +3,17 @@ #include -#include #include +#include -#include -#include -#include -#include -#include -#include #include -#include -#include - #include #include #include #include -#include -#include #include -#include -#include #include #include @@ -75,177 +62,6 @@ static RocksDBOptions getOptionsFromConfig(const Poco::Util::AbstractConfigurati return options; } -// returns keys may be filter by condition -static bool traverseASTFilter( - const String & primary_key, const DataTypePtr & primary_key_type, const ASTPtr & elem, const PreparedSets & sets, const ContextPtr & context, FieldVectorPtr & res) -{ - const auto * function = elem->as(); - if (!function) - return false; - - if (function->name == "and") - { - // one child has the key filter condition is ok - for (const auto & child : function->arguments->children) - if (traverseASTFilter(primary_key, primary_key_type, child, sets, context, res)) - return true; - return false; - } - else if (function->name == "or") - { - // make sure every child has the key filter condition - for (const auto & child : function->arguments->children) - if (!traverseASTFilter(primary_key, primary_key_type, child, sets, context, res)) - return false; - return true; - } - else if (function->name == "equals" || function->name == "in") - { - const auto & args = function->arguments->as(); - const ASTIdentifier * ident; - std::shared_ptr value; - - if (args.children.size() != 2) - return false; - - if (function->name == "in") - { - ident = args.children.at(0)->as(); - if (!ident) - return false; - - if (ident->name() != primary_key) - return false; - value = args.children.at(1); - - PreparedSetKey set_key; - if ((value->as() || value->as())) - set_key = PreparedSetKey::forSubquery(*value); - else - set_key = PreparedSetKey::forLiteral(*value, {primary_key_type}); - - auto set_it = sets.find(set_key); - if (set_it == sets.end()) - return false; - SetPtr prepared_set = set_it->second; - - if (!prepared_set->hasExplicitSetElements()) - return false; - - prepared_set->checkColumnsNumber(1); - const auto & set_column = *prepared_set->getSetElements()[0]; - for (size_t row = 0; row < set_column.size(); ++row) - res->push_back(set_column[row]); - return true; - } - else - { - if ((ident = args.children.at(0)->as())) - value = args.children.at(1); - else if ((ident = args.children.at(1)->as())) - value = args.children.at(0); - else - return false; - - if (ident->name() != primary_key) - return false; - - const auto node = evaluateConstantExpressionAsLiteral(value, context); - /// function->name == "equals" - if (const auto * literal = node->as()) - { - auto converted_field = convertFieldToType(literal->value, *primary_key_type); - if (!converted_field.isNull()) - res->push_back(converted_field); - return true; - } - } - } - return false; -} - -/** Retrieve from the query a condition of the form `key = 'key'`, `key in ('xxx_'), from conjunctions in the WHERE clause. - * TODO support key like search - */ -static std::pair getFilterKeys( - const String & primary_key, const DataTypePtr & primary_key_type, const SelectQueryInfo & query_info, const ContextPtr & context) -{ - const auto & select = query_info.query->as(); - if (!select.where()) - return {{}, true}; - - FieldVectorPtr res = std::make_shared(); - auto matched_keys = traverseASTFilter(primary_key, primary_key_type, select.where(), query_info.sets, context, res); - return std::make_pair(res, !matched_keys); -} - -template -static void fillColumns(const K & key, const V & value, size_t key_pos, const Block & header, MutableColumns & columns) -{ - ReadBufferFromString key_buffer(key); - ReadBufferFromString value_buffer(value); - for (size_t i = 0; i < header.columns(); ++i) - { - const auto & serialization = header.getByPosition(i).type->getDefaultSerialization(); - serialization->deserializeBinary(*columns[i], i == key_pos ? key_buffer : value_buffer); - } -} - -static std::vector serializeKeysToRawString( - FieldVector::const_iterator & it, - FieldVector::const_iterator end, - DataTypePtr key_column_type, - size_t max_block_size) -{ - size_t num_keys = end - it; - - std::vector result; - result.reserve(num_keys); - - size_t rows_processed = 0; - while (it < end && (max_block_size == 0 || rows_processed < max_block_size)) - { - std::string & serialized_key = result.emplace_back(); - WriteBufferFromString wb(serialized_key); - key_column_type->getDefaultSerialization()->serializeBinary(*it, wb); - wb.finalize(); - - ++it; - ++rows_processed; - } - return result; -} - -static std::vector serializeKeysToRawString(const ColumnWithTypeAndName & keys) -{ - if (!keys.column) - return {}; - - size_t num_keys = keys.column->size(); - std::vector result; - result.reserve(num_keys); - - for (size_t i = 0; i < num_keys; ++i) - { - std::string & serialized_key = result.emplace_back(); - WriteBufferFromString wb(serialized_key); - Field field; - keys.column->get(i, field); - /// TODO(@vdimir): use serializeBinaryBulk - keys.type->getDefaultSerialization()->serializeBinary(field, wb); - wb.finalize(); - } - return result; -} - -/// In current implementation rocks db can have key with only one column. -static size_t getPrimaryKeyPos(const Block & header, const Names & primary_key) -{ - if (primary_key.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "RocksDB: only one primary key is supported"); - return header.getPositionByName(primary_key[0]); -} - class EmbeddedRocksDBSource : public ISource { public: diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp new file mode 100644 index 00000000000..1208ac31477 --- /dev/null +++ b/src/Storages/StorageKeeperMap.cpp @@ -0,0 +1,329 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "Parsers/ASTExpressionList.h" +#include "Parsers/ASTFunction.h" +#include +#include +#include "Parsers/ASTSelectQuery.h" +#include "Storages/MergeTree/IMergeTreeDataPart.h" +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; +} + +StorageKeeperMap::StorageKeeperMap(std::string_view keeper_path_, ContextPtr context, const StorageID & table_id) + : IStorage(table_id), keeper_path(keeper_path_), zookeeper_client(context->getZooKeeper()->startNewSession()) +{ + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(ColumnsDescription{getNamesAndTypes()}); + setInMemoryMetadata(storage_metadata); + + if (keeper_path.empty()) + throw Exception("keeper_path should not be empty", ErrorCodes::BAD_ARGUMENTS); + if (!keeper_path.starts_with('/')) + throw Exception("keeper_path should start with '/'", ErrorCodes::BAD_ARGUMENTS); + + if (keeper_path != "/") + { + LOG_TRACE(&Poco::Logger::get("StorageKeeperMap"), "Creating root path {}", keeper_path); + + size_t cur_pos = 0; + do + { + size_t search_start = cur_pos + 1; + cur_pos = keeper_path.find('/', search_start); + if (search_start == cur_pos) + throw Exception("keeper_path is invalid, contains subsequent '/'", ErrorCodes::BAD_ARGUMENTS); + + auto path = keeper_path.substr(0, cur_pos); + auto status = getClient()->tryCreate(path, "", zkutil::CreateMode::Persistent); + if (status != Coordination::Error::ZOK && status != Coordination::Error::ZNODEEXISTS) + throw zkutil::KeeperException(status, path); + } while (cur_pos != std::string_view::npos); + } +} + +NamesAndTypesList StorageKeeperMap::getNamesAndTypes() +{ + return {{"key", std::make_shared()}, {"value", std::make_shared()}}; +} + +class StorageKeeperMapSink : public SinkToStorage +{ + StorageKeeperMap & storage; + std::unordered_map new_values; + +public: + StorageKeeperMapSink(const Block & header, StorageKeeperMap & storage_) : SinkToStorage(header), storage(storage_) { } + + std::string getName() const override { return "StorageKeeperMapSink"; } + + void consume(Chunk chunk) override + { + auto block = getHeader().cloneWithColumns(chunk.getColumns()); + + size_t rows = block.rows(); + for (size_t i = 0; i < rows; ++i) + { + std::string key = block.getByPosition(0).column->getDataAt(i).toString(); + + if (key.find('/') != std::string::npos) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key cannot contain '/'. Key: '{}'", key); + + std::string value = block.getByPosition(1).column->getDataAt(i).toString(); + + new_values[std::move(key)] = std::move(value); + } + } + + void onFinish() override + { + auto & zookeeper = storage.getClient(); + Coordination::Requests requests; + for (const auto & [key, value] : new_values) + { + auto path = fmt::format("{}/{}", storage.rootKeeperPath(), key); + + if (zookeeper->exists(path)) + requests.push_back(zkutil::makeSetRequest(path, value, -1)); + else + requests.push_back(zkutil::makeCreateRequest(path, value, zkutil::CreateMode::Persistent)); + } + + zookeeper->multi(requests); + } +}; + +enum class FilterType +{ + EXACT = 0 +}; + +struct KeyFilter +{ + std::string filter; + FilterType type; +}; + +class StorageKeeperMapSource : public ISource +{ + StorageKeeperMap & storage; + std::vector keys; + size_t current_idx = 0; + Block sample_block; + Names column_names; + bool has_value_column{false}; + size_t max_block_size; + std::optional filter; + + Chunk generateSingleKey() + { + assert(filter && filter->type == FilterType::EXACT); + static bool processed = false; + + if (processed) + return {}; + + auto zookeeper = storage.getClient(); + + std::string value; + auto path = fmt::format("{}/{}", storage.rootKeeperPath(), filter->filter); + auto res = zookeeper->tryGet(path, value); + if (!res) + return {}; + + MutableColumns columns(sample_block.cloneEmptyColumns()); + insertRowForKey(columns, filter->filter, value); + processed = true; + + return Chunk{std::move(columns), 1}; + } + + bool insertRowForKey(MutableColumns & columns, const std::string & key, const std::string & value) + { + + for (size_t column_index = 0; column_index < column_names.size(); ++column_index) + { + if (column_names[column_index] == "key") + assert_cast(*columns[column_index]).insertData(key.data(), key.size()); + else if (column_names[column_index] == "value") + assert_cast(*columns[column_index]).insertData(value.data(), value.size()); + } + return true; + } + +public: + StorageKeeperMapSource(const Block & sample_block_, StorageKeeperMap & storage_, size_t max_block_size_, std::optional filter_) + : ISource(sample_block_) + , storage(storage_) + , sample_block(sample_block_.cloneEmpty()) + , column_names(sample_block_.getNames()) + , max_block_size(max_block_size_) + , filter(std::move(filter_)) + { + has_value_column = std::any_of(column_names.begin(), column_names.end(), [](const auto & name) { return name == "value"; }); + + // TODO(antonio2368): Do it lazily in generate + if (!filter || filter->type != FilterType::EXACT) + { + auto zookeeper = storage.getClient(); + keys = zookeeper->getChildren(storage.rootKeeperPath()); + } + } + + std::string getName() const override { return "StorageKeeperMapSource"; } + + Chunk generate() override + { + if (filter && filter->type == FilterType::EXACT) + return generateSingleKey(); + + auto zookeeper = storage.getClient(); + + MutableColumns columns(sample_block.cloneEmptyColumns()); + size_t num_rows = 0; + for (; num_rows < max_block_size && current_idx != keys.size(); ++current_idx) + { + const auto & key = keys[current_idx]; + std::string value; + if (has_value_column) + { + auto path = fmt::format("{}/{}", storage.rootKeeperPath(), key); + auto res = zookeeper->tryGet(path, value); + if (!res) + continue; + } + + if (insertRowForKey(columns, key, value)) + ++num_rows; + } + + if (num_rows == 0) + return {}; + + return {std::move(columns), num_rows}; + } +}; + +std::optional tryGetKeyFilter(const IAST & elem, const ContextPtr context) +{ + const auto * function = elem.as(); + if (!function) + return std::nullopt; + + if (function->name != "equals") + return std::nullopt; + + const auto & args = function->arguments->as(); + const ASTIdentifier * ident; + ASTPtr value; + if ((ident = args.children.at(0)->as())) + value = args.children.at(1); + else if ((ident = args.children.at(1)->as())) + value = args.children.at(0); + else + return std::nullopt; + + if (ident->name() != "key") + return std::nullopt; + + auto evaluated = evaluateConstantExpressionAsLiteral(value, context); + const auto * literal = evaluated->as(); + if (!literal) + return std::nullopt; + + if (literal->value.getType() != Field::Types::String) + return std::nullopt; + + return KeyFilter{literal->value.safeGet(), FilterType::EXACT}; +} + +Pipe StorageKeeperMap::read( + const Names & column_names, + const StorageSnapshotPtr & /*storage_snapshot*/, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum /*processed_stage*/, + size_t max_block_size, + unsigned /*num_streams*/) +{ + Block sample_block; + for (const std::string & column_name : column_names) + { + sample_block.insert({std::make_shared(), column_name}); + } + + const auto & select = query_info.query->as(); + std::optional key_filter; + if (select.where()) + key_filter = tryGetKeyFilter(*select.where(), context); + + return Pipe(std::make_shared(sample_block, *this, max_block_size, std::move(key_filter))); +} + +SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) +{ + auto columns = getNamesAndTypes(); + Block write_header; + for (const auto & [name, type] : columns) + { + write_header.insert(ColumnWithTypeAndName(type, name)); + } + + return std::make_shared(write_header, *this); +} + +zkutil::ZooKeeperPtr & StorageKeeperMap::getClient() +{ + if (zookeeper_client->expired()) + zookeeper_client = zookeeper_client->startNewSession(); + + return zookeeper_client; +} + +const std::string & StorageKeeperMap::rootKeeperPath() const +{ + return keeper_path; +} + +void registerStorageKeeperMap(StorageFactory & factory) +{ + factory.registerStorage( + "KeeperMap", + [](const StorageFactory::Arguments & args) + { + if (!args.attach && !args.columns.empty()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Storage KeeperMap does not accept column definition as it has predefined columns (key String, value String)"); + + ASTs & engine_args = args.engine_args; + if (engine_args.empty() || engine_args.size() > 1) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage KeeperMap requires 1 argument: " + "keeper_path, path in the Keeper where the values will be stored"); + + auto keeper_path = checkAndGetLiteralArgument(engine_args[0], "keeper_path"); + + return std::make_shared(keeper_path, args.getContext(), args.table_id); + }, + {}); +} + +} diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h new file mode 100644 index 00000000000..32542a6ecf6 --- /dev/null +++ b/src/Storages/StorageKeeperMap.h @@ -0,0 +1,52 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +// KV store using (Zoo|CH)Keeper +class StorageKeeperMap final : public IStorage +{ +public: + // TODO(antonio2368): add setting to control creating if keeper_path doesn't exist + StorageKeeperMap( + std::string_view keeper_path_, + ContextPtr context, + const StorageID & table_id + ); + + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + + SinkToStoragePtr write( + const ASTPtr & query, + const StorageMetadataPtr & /*metadata_snapshot*/, + ContextPtr context) override; + + std::string getName() const override + { + return "KeeperMap"; + } + + static NamesAndTypesList getNamesAndTypes(); + + zkutil::ZooKeeperPtr & getClient(); + + const std::string & rootKeeperPath() const; +private: + + std::string keeper_path; + zkutil::ZooKeeperPtr zookeeper_client; +}; + +} diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 575b3de7ae2..055270be4ae 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -88,6 +88,7 @@ void registerStorageFileLog(StorageFactory & factory); void registerStorageSQLite(StorageFactory & factory); #endif +void registerStorageKeeperMap(StorageFactory & factory); void registerStorages() { @@ -171,6 +172,8 @@ void registerStorages() #if USE_SQLITE registerStorageSQLite(factory); #endif + + registerStorageKeeperMap(factory); } } From 034807030d199a3cac5bd3e9af292c93d0cc65ac Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 27 Jul 2022 13:20:45 +0000 Subject: [PATCH 02/43] Support multiple columns --- src/Storages/KVStorageUtils.cpp | 2 +- .../RocksDB/StorageEmbeddedRocksDB.cpp | 1 - src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 1 + src/Storages/StorageKeeperMap.cpp | 587 ++++++++++-------- src/Storages/StorageKeeperMap.h | 52 +- 5 files changed, 369 insertions(+), 274 deletions(-) diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index 8c99d593f40..91b7e7feda6 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -168,7 +168,7 @@ std::vector serializeKeysToRawString(const ColumnWithTypeAndName & size_t getPrimaryKeyPos(const Block & header, const Names & primary_key) { if (primary_key.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "RocksDB: only one primary key is supported"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Only one primary key is supported"); return header.getPositionByName(primary_key[0]); } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index dda27c281cb..9df60ede024 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -404,7 +404,6 @@ Chunk StorageEmbeddedRocksDB::getBySerializedKeys( for (const auto & key : keys) slices_keys.emplace_back(key); - auto statuses = multiGet(slices_keys, values); if (null_map) { diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 62c9a0eeae7..169b6c1e1bc 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 1208ac31477..11c733dd0bb 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -1,37 +1,202 @@ +#include + #include + +#include + #include + +#include + +#include +#include +#include +#include +#include + #include #include + +#include +#include +#include #include -#include -#include +#include +#include + #include #include -#include "Parsers/ASTExpressionList.h" -#include "Parsers/ASTFunction.h" -#include -#include -#include "Parsers/ASTSelectQuery.h" -#include "Storages/MergeTree/IMergeTreeDataPart.h" -#include -#include -#include +#include + +#include +#include +#include +#include +#include namespace DB { namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int BAD_ARGUMENTS; +extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +extern const int BAD_ARGUMENTS; +extern const int KEEPER_EXCEPTION; } -StorageKeeperMap::StorageKeeperMap(std::string_view keeper_path_, ContextPtr context, const StorageID & table_id) - : IStorage(table_id), keeper_path(keeper_path_), zookeeper_client(context->getZooKeeper()->startNewSession()) +namespace { - StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(ColumnsDescription{getNamesAndTypes()}); - setInMemoryMetadata(storage_metadata); + +std::string base64Encode(const std::string & decoded) +{ + std::ostringstream ostr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + ostr.exceptions(std::ios::failbit); + Poco::Base64Encoder encoder(ostr, Poco::BASE64_URL_ENCODING); + encoder.rdbuf()->setLineLength(0); + encoder << decoded; + encoder.close(); + return ostr.str(); +} + +std::string base64Decode(const std::string & encoded) +{ + std::string decoded; + Poco::MemoryInputStream istr(encoded.data(), encoded.size()); + Poco::Base64Decoder decoder(istr, Poco::BASE64_URL_ENCODING); + Poco::StreamCopier::copyToString(decoder, decoded); + return decoded; +} + +} + +class StorageKeeperMapSink : public SinkToStorage +{ + StorageKeeperMap & storage; + std::unordered_map new_values; + size_t primary_key_pos; + +public: + StorageKeeperMapSink(StorageKeeperMap & storage_, const StorageMetadataPtr & metadata_snapshot) + : SinkToStorage(metadata_snapshot->getSampleBlock()), storage(storage_) + { + auto primary_key = storage.getPrimaryKey(); + assert(primary_key.size() == 1); + primary_key_pos = getHeader().getPositionByName(storage.getPrimaryKey()[0]); + } + + std::string getName() const override { return "StorageKeeperMapSink"; } + + void consume(Chunk chunk) override + { + auto rows = chunk.getNumRows(); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + + WriteBufferFromOwnString wb_key; + WriteBufferFromOwnString wb_value; + + for (size_t i = 0; i < rows; ++i) + { + wb_key.restart(); + wb_value.restart(); + + size_t idx = 0; + for (const auto & elem : block) + { + elem.type->getDefaultSerialization()->serializeBinary(*elem.column, i, idx == primary_key_pos ? wb_key : wb_value); + ++idx; + } + + auto key = base64Encode(wb_key.str()); + new_values[std::move(key)] = std::move(wb_value.str()); + } + } + + void onFinish() override + { + auto & zookeeper = storage.getClient(); + Coordination::Requests requests; + for (const auto & [key, value] : new_values) + { + auto path = storage.fullPathForKey(key); + + if (zookeeper->exists(path)) + requests.push_back(zkutil::makeSetRequest(path, value, -1)); + else + requests.push_back(zkutil::makeCreateRequest(path, value, zkutil::CreateMode::Persistent)); + } + + zookeeper->multi(requests); + } +}; + +template +class StorageKeeperMapSource : public ISource +{ + const StorageKeeperMap & storage; + size_t max_block_size; + + using KeyContainerPtr = std::shared_ptr; + KeyContainerPtr container; + using KeyContainerIter = typename KeyContainer::const_iterator; + KeyContainerIter it; + KeyContainerIter end; + +public: + StorageKeeperMapSource( + const StorageKeeperMap & storage_, + const Block & header, + size_t max_block_size_, + KeyContainerPtr container_, + KeyContainerIter begin_, + KeyContainerIter end_) + : ISource(header), storage(storage_), max_block_size(max_block_size_), container(std::move(container_)), it(begin_), end(end_) + { + } + + std::string getName() const override { return "StorageKeeperMapSource"; } + + Chunk generate() override + { + if (it >= end) + { + it = {}; + return {}; + } + + using KeyType = typename KeyContainer::value_type; + if constexpr (std::same_as) + { + const auto & sample_block = getPort().getHeader(); + const auto & key_column_type = sample_block.getByName(storage.getPrimaryKey().at(0)).type; + auto raw_keys = serializeKeysToRawString(it, end, key_column_type, max_block_size); + + for (auto & raw_key : raw_keys) + raw_key = base64Encode(raw_key); + + return storage.getBySerializedKeys(raw_keys, nullptr); + } + else + { + size_t elem_num = std::min(max_block_size, static_cast(end - it)); + auto chunk = storage.getBySerializedKeys(std::span{it, it + elem_num}, nullptr); + it += elem_num; + return chunk; + } + } +}; + +StorageKeeperMap::StorageKeeperMap( + ContextPtr context, + const StorageID & table_id, + const StorageInMemoryMetadata & metadata, + std::string_view primary_key_, + std::string_view keeper_path_) + : IKeyValueStorage(table_id) + , keeper_path(keeper_path_) + , primary_key(primary_key_) + , zookeeper_client(context->getZooKeeper()->startNewSession()) +{ + setInMemoryMetadata(metadata); if (keeper_path.empty()) throw Exception("keeper_path should not be empty", ErrorCodes::BAD_ARGUMENTS); @@ -58,237 +223,66 @@ StorageKeeperMap::StorageKeeperMap(std::string_view keeper_path_, ContextPtr con } } -NamesAndTypesList StorageKeeperMap::getNamesAndTypes() -{ - return {{"key", std::make_shared()}, {"value", std::make_shared()}}; -} - -class StorageKeeperMapSink : public SinkToStorage -{ - StorageKeeperMap & storage; - std::unordered_map new_values; - -public: - StorageKeeperMapSink(const Block & header, StorageKeeperMap & storage_) : SinkToStorage(header), storage(storage_) { } - - std::string getName() const override { return "StorageKeeperMapSink"; } - - void consume(Chunk chunk) override - { - auto block = getHeader().cloneWithColumns(chunk.getColumns()); - - size_t rows = block.rows(); - for (size_t i = 0; i < rows; ++i) - { - std::string key = block.getByPosition(0).column->getDataAt(i).toString(); - - if (key.find('/') != std::string::npos) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key cannot contain '/'. Key: '{}'", key); - - std::string value = block.getByPosition(1).column->getDataAt(i).toString(); - - new_values[std::move(key)] = std::move(value); - } - } - - void onFinish() override - { - auto & zookeeper = storage.getClient(); - Coordination::Requests requests; - for (const auto & [key, value] : new_values) - { - auto path = fmt::format("{}/{}", storage.rootKeeperPath(), key); - - if (zookeeper->exists(path)) - requests.push_back(zkutil::makeSetRequest(path, value, -1)); - else - requests.push_back(zkutil::makeCreateRequest(path, value, zkutil::CreateMode::Persistent)); - } - - zookeeper->multi(requests); - } -}; - -enum class FilterType -{ - EXACT = 0 -}; - -struct KeyFilter -{ - std::string filter; - FilterType type; -}; - -class StorageKeeperMapSource : public ISource -{ - StorageKeeperMap & storage; - std::vector keys; - size_t current_idx = 0; - Block sample_block; - Names column_names; - bool has_value_column{false}; - size_t max_block_size; - std::optional filter; - - Chunk generateSingleKey() - { - assert(filter && filter->type == FilterType::EXACT); - static bool processed = false; - - if (processed) - return {}; - - auto zookeeper = storage.getClient(); - - std::string value; - auto path = fmt::format("{}/{}", storage.rootKeeperPath(), filter->filter); - auto res = zookeeper->tryGet(path, value); - if (!res) - return {}; - - MutableColumns columns(sample_block.cloneEmptyColumns()); - insertRowForKey(columns, filter->filter, value); - processed = true; - - return Chunk{std::move(columns), 1}; - } - - bool insertRowForKey(MutableColumns & columns, const std::string & key, const std::string & value) - { - - for (size_t column_index = 0; column_index < column_names.size(); ++column_index) - { - if (column_names[column_index] == "key") - assert_cast(*columns[column_index]).insertData(key.data(), key.size()); - else if (column_names[column_index] == "value") - assert_cast(*columns[column_index]).insertData(value.data(), value.size()); - } - return true; - } - -public: - StorageKeeperMapSource(const Block & sample_block_, StorageKeeperMap & storage_, size_t max_block_size_, std::optional filter_) - : ISource(sample_block_) - , storage(storage_) - , sample_block(sample_block_.cloneEmpty()) - , column_names(sample_block_.getNames()) - , max_block_size(max_block_size_) - , filter(std::move(filter_)) - { - has_value_column = std::any_of(column_names.begin(), column_names.end(), [](const auto & name) { return name == "value"; }); - - // TODO(antonio2368): Do it lazily in generate - if (!filter || filter->type != FilterType::EXACT) - { - auto zookeeper = storage.getClient(); - keys = zookeeper->getChildren(storage.rootKeeperPath()); - } - } - - std::string getName() const override { return "StorageKeeperMapSource"; } - - Chunk generate() override - { - if (filter && filter->type == FilterType::EXACT) - return generateSingleKey(); - - auto zookeeper = storage.getClient(); - - MutableColumns columns(sample_block.cloneEmptyColumns()); - size_t num_rows = 0; - for (; num_rows < max_block_size && current_idx != keys.size(); ++current_idx) - { - const auto & key = keys[current_idx]; - std::string value; - if (has_value_column) - { - auto path = fmt::format("{}/{}", storage.rootKeeperPath(), key); - auto res = zookeeper->tryGet(path, value); - if (!res) - continue; - } - - if (insertRowForKey(columns, key, value)) - ++num_rows; - } - - if (num_rows == 0) - return {}; - - return {std::move(columns), num_rows}; - } -}; - -std::optional tryGetKeyFilter(const IAST & elem, const ContextPtr context) -{ - const auto * function = elem.as(); - if (!function) - return std::nullopt; - - if (function->name != "equals") - return std::nullopt; - - const auto & args = function->arguments->as(); - const ASTIdentifier * ident; - ASTPtr value; - if ((ident = args.children.at(0)->as())) - value = args.children.at(1); - else if ((ident = args.children.at(1)->as())) - value = args.children.at(0); - else - return std::nullopt; - - if (ident->name() != "key") - return std::nullopt; - - auto evaluated = evaluateConstantExpressionAsLiteral(value, context); - const auto * literal = evaluated->as(); - if (!literal) - return std::nullopt; - - if (literal->value.getType() != Field::Types::String) - return std::nullopt; - - return KeyFilter{literal->value.safeGet(), FilterType::EXACT}; -} Pipe StorageKeeperMap::read( const Names & column_names, - const StorageSnapshotPtr & /*storage_snapshot*/, + const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, - ContextPtr context, + ContextPtr /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - unsigned /*num_streams*/) + unsigned num_streams) { - Block sample_block; - for (const std::string & column_name : column_names) + storage_snapshot->check(column_names); + + FieldVectorPtr filtered_keys; + bool all_scan; + + Block sample_block = storage_snapshot->metadata->getSampleBlock(); + auto primary_key_type = sample_block.getByName(primary_key).type; + std::tie(filtered_keys, all_scan) = getFilterKeys(primary_key, primary_key_type, query_info); + + const auto process_keys = [&](KeyContainerPtr keys) -> Pipe { - sample_block.insert({std::make_shared(), column_name}); - } + if (keys->empty()) + return {}; - const auto & select = query_info.query->as(); - std::optional key_filter; - if (select.where()) - key_filter = tryGetKeyFilter(*select.where(), context); + ::sort(keys->begin(), keys->end()); + keys->erase(std::unique(keys->begin(), keys->end()), keys->end()); - return Pipe(std::make_shared(sample_block, *this, max_block_size, std::move(key_filter))); + Pipes pipes; + + size_t num_keys = keys->size(); + size_t num_threads = std::min(num_streams, keys->size()); + + assert(num_keys <= std::numeric_limits::max()); + assert(num_threads <= std::numeric_limits::max()); + + for (size_t thread_idx = 0; thread_idx < num_threads; ++thread_idx) + { + size_t begin = num_keys * thread_idx / num_threads; + size_t end = num_keys * (thread_idx + 1) / num_threads; + + using KeyContainer = typename KeyContainerPtr::element_type; + pipes.emplace_back(std::make_shared>( + *this, sample_block, max_block_size, keys, keys->begin() + begin, keys->begin() + end)); + } + return Pipe::unitePipes(std::move(pipes)); + }; + + auto & client = getClient(); + if (all_scan) + return process_keys(std::make_shared>(client->getChildren(keeper_path))); + + return process_keys(std::move(filtered_keys)); } -SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) +SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) { - auto columns = getNamesAndTypes(); - Block write_header; - for (const auto & [name, type] : columns) - { - write_header.insert(ColumnWithTypeAndName(type, name)); - } - - return std::make_shared(write_header, *this); + return std::make_shared(*this, metadata_snapshot); } -zkutil::ZooKeeperPtr & StorageKeeperMap::getClient() +zkutil::ZooKeeperPtr & StorageKeeperMap::getClient() const { if (zookeeper_client->expired()) zookeeper_client = zookeeper_client->startNewSession(); @@ -301,29 +295,118 @@ const std::string & StorageKeeperMap::rootKeeperPath() const return keeper_path; } +std::string StorageKeeperMap::fullPathForKey(const std::string_view key) const +{ + return fmt::format("{}/{}", keeper_path, key); +} + +Chunk StorageKeeperMap::getByKeys(const ColumnsWithTypeAndName & keys, PaddedPODArray & null_map) const +{ + if (keys.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "StorageKeeperMap supports only one key, got: {}", keys.size()); + + auto raw_keys = serializeKeysToRawString(keys[0]); + + if (raw_keys.size() != keys[0].column->size()) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Assertion failed: {} != {}", raw_keys.size(), keys[0].column->size()); + + return getBySerializedKeys(raw_keys, &null_map); +} + +Chunk StorageKeeperMap::getBySerializedKeys(const std::span keys, PaddedPODArray * null_map) const +{ + Block sample_block = getInMemoryMetadataPtr()->getSampleBlock(); + MutableColumns columns = sample_block.cloneEmptyColumns(); + size_t primary_key_pos = getPrimaryKeyPos(sample_block, getPrimaryKey()); + + if (null_map) + { + null_map->clear(); + null_map->resize_fill(keys.size(), 1); + } + + auto client = getClient(); + + std::vector> values; + values.reserve(keys.size()); + + for (const auto & key : keys) + { + values.emplace_back(client->asyncTryGet(fullPathForKey(key))); + } + + auto wait_until = std::chrono::system_clock::now() + std::chrono::milliseconds(Coordination::DEFAULT_OPERATION_TIMEOUT_MS); + + for (size_t i = 0; i < keys.size(); ++i) + { + auto & value = values[i]; + if (value.wait_until(wait_until) != std::future_status::ready) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Failed to fetch values: timeout"); + + auto response = value.get(); + Coordination::Error code = response.error; + + if (code == Coordination::Error::ZOK) + { + fillColumns(base64Decode(keys[i]), response.data, primary_key_pos, sample_block, columns); + } + else if (code == Coordination::Error::ZNONODE) + { + if (null_map) + { + (*null_map)[i] = 0; + for (size_t col_idx = 0; col_idx < sample_block.columns(); ++col_idx) + columns[col_idx]->insert(sample_block.getByPosition(col_idx).type->getDefault()); + } + } + else + { + throw DB::Exception(ErrorCodes::KEEPER_EXCEPTION, "Failed to fetch value: {}", code); + } + } + + size_t num_rows = columns.at(0)->size(); + return Chunk(std::move(columns), num_rows); +} + +namespace +{ +StoragePtr create(const StorageFactory::Arguments & args) +{ + ASTs & engine_args = args.engine_args; + if (engine_args.empty() || engine_args.size() > 1) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage KeeperMap requires 1 argument: " + "keeper_path, path in the Keeper where the values will be stored"); + + auto keeper_path = checkAndGetLiteralArgument(engine_args[0], "keeper_path"); + + StorageInMemoryMetadata metadata; + metadata.setColumns(args.columns); + metadata.setConstraints(args.constraints); + + if (!args.storage_def->primary_key) + throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); + + metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.getContext()); + auto primary_key_names = metadata.getColumnsRequiredForPrimaryKey(); + if (primary_key_names.size() != 1) + throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); + + return std::make_shared(args.getContext(), args.table_id, metadata, primary_key_names[0], keeper_path); +} +} + void registerStorageKeeperMap(StorageFactory & factory) { factory.registerStorage( "KeeperMap", - [](const StorageFactory::Arguments & args) + create, { - if (!args.attach && !args.columns.empty()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Storage KeeperMap does not accept column definition as it has predefined columns (key String, value String)"); - - ASTs & engine_args = args.engine_args; - if (engine_args.empty() || engine_args.size() > 1) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage KeeperMap requires 1 argument: " - "keeper_path, path in the Keeper where the values will be stored"); - - auto keeper_path = checkAndGetLiteralArgument(engine_args[0], "keeper_path"); - - return std::make_shared(keeper_path, args.getContext(), args.table_id); - }, - {}); + .supports_sort_order = true, + .supports_parallel_insert = true, + }); } } diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 32542a6ecf6..bde76e958ba 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -1,23 +1,29 @@ #pragma once -#include #include #include +#include #include +#include +#include "Common/PODArray_fwd.h" +#include + +#include namespace DB { -// KV store using (Zoo|CH)Keeper -class StorageKeeperMap final : public IStorage +// KV store using (Zoo|CH)Keeper +class StorageKeeperMap final : public IKeyValueStorage { public: // TODO(antonio2368): add setting to control creating if keeper_path doesn't exist StorageKeeperMap( - std::string_view keeper_path_, - ContextPtr context, - const StorageID & table_id - ); + ContextPtr context, + const StorageID & table_id, + const StorageInMemoryMetadata & metadata, + std::string_view primary_key_, + std::string_view keeper_path_); Pipe read( const Names & column_names, @@ -27,26 +33,32 @@ public: QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; - - SinkToStoragePtr write( - const ASTPtr & query, - const StorageMetadataPtr & /*metadata_snapshot*/, - ContextPtr context) override; - std::string getName() const override + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; + + std::string getName() const override { return "KeeperMap"; } + Names getPrimaryKey() const override { return {primary_key}; } + + Chunk getByKeys(const ColumnsWithTypeAndName & keys, PaddedPODArray & null_map) const override; + Chunk getBySerializedKeys(std::span keys, PaddedPODArray * null_map) const; + + bool supportsParallelInsert() const override { return true; } + bool supportsIndexForIn() const override { return true; } + bool mayBenefitFromIndexForIn( + const ASTPtr & node, ContextPtr /*query_context*/, const StorageMetadataPtr & /*metadata_snapshot*/) const override { - return "KeeperMap"; + return node->getColumnName() == primary_key; } - static NamesAndTypesList getNamesAndTypes(); - - zkutil::ZooKeeperPtr & getClient(); - + zkutil::ZooKeeperPtr & getClient() const; const std::string & rootKeeperPath() const; -private: + std::string fullPathForKey(std::string_view key) const; +private: std::string keeper_path; - zkutil::ZooKeeperPtr zookeeper_client; + std::string primary_key; + + mutable zkutil::ZooKeeperPtr zookeeper_client; }; } From 85927c18c06624f4ae42d76ac0f108ceae79ee4c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 3 Aug 2022 13:34:14 +0000 Subject: [PATCH 03/43] Add more engine arguments --- src/Storages/StorageKeeperMap.cpp | 99 ++++++++++++++++++++++--------- src/Storages/StorageKeeperMap.h | 4 +- 2 files changed, 75 insertions(+), 28 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 11c733dd0bb..8e297e63273 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -39,9 +39,9 @@ namespace DB namespace ErrorCodes { -extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -extern const int BAD_ARGUMENTS; -extern const int KEEPER_EXCEPTION; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; + extern const int KEEPER_EXCEPTION; } namespace @@ -67,6 +67,8 @@ std::string base64Decode(const std::string & encoded) return decoded; } +constexpr std::string_view default_host = "default"; + } class StorageKeeperMapSink : public SinkToStorage @@ -185,16 +187,28 @@ public: } }; +namespace +{ + +zkutil::ZooKeeperPtr getZooKeeperClient(const std::string & hosts, const ContextPtr & context) +{ + if (hosts == default_host) + return context->getZooKeeper()->startNewSession(); + + return std::make_shared(hosts); +} + +} + StorageKeeperMap::StorageKeeperMap( ContextPtr context, const StorageID & table_id, const StorageInMemoryMetadata & metadata, std::string_view primary_key_, - std::string_view keeper_path_) - : IKeyValueStorage(table_id) - , keeper_path(keeper_path_) - , primary_key(primary_key_) - , zookeeper_client(context->getZooKeeper()->startNewSession()) + std::string_view keeper_path_, + const std::string & hosts, + bool create_missing_root_path) + : IKeyValueStorage(table_id), keeper_path(keeper_path_), primary_key(primary_key_), zookeeper_client(getZooKeeperClient(hosts, context)) { setInMemoryMetadata(metadata); @@ -203,23 +217,34 @@ StorageKeeperMap::StorageKeeperMap( if (!keeper_path.starts_with('/')) throw Exception("keeper_path should start with '/'", ErrorCodes::BAD_ARGUMENTS); - if (keeper_path != "/") + auto client = getClient(); + if (keeper_path != "/" && !client->exists(keeper_path)) { - LOG_TRACE(&Poco::Logger::get("StorageKeeperMap"), "Creating root path {}", keeper_path); - - size_t cur_pos = 0; - do + if (!create_missing_root_path) { - size_t search_start = cur_pos + 1; - cur_pos = keeper_path.find('/', search_start); - if (search_start == cur_pos) - throw Exception("keeper_path is invalid, contains subsequent '/'", ErrorCodes::BAD_ARGUMENTS); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Path '{}' doesn't exist. Please create it or set 'create_missing_root_path' to true'", + keeper_path_); + } + else + { + LOG_TRACE(&Poco::Logger::get("StorageKeeperMap"), "Creating root path {}", keeper_path); - auto path = keeper_path.substr(0, cur_pos); - auto status = getClient()->tryCreate(path, "", zkutil::CreateMode::Persistent); - if (status != Coordination::Error::ZOK && status != Coordination::Error::ZNODEEXISTS) - throw zkutil::KeeperException(status, path); - } while (cur_pos != std::string_view::npos); + size_t cur_pos = 0; + do + { + size_t search_start = cur_pos + 1; + cur_pos = keeper_path.find('/', search_start); + if (search_start == cur_pos) + throw Exception("keeper_path is invalid, contains subsequent '/'", ErrorCodes::BAD_ARGUMENTS); + + auto path = keeper_path.substr(0, cur_pos); + auto status = client->tryCreate(path, "", zkutil::CreateMode::Persistent); + if (status != Coordination::Error::ZOK && status != Coordination::Error::ZNODEEXISTS) + throw zkutil::KeeperException(status, path); + } while (cur_pos != std::string_view::npos); + } } } @@ -285,7 +310,10 @@ SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const Storage zkutil::ZooKeeperPtr & StorageKeeperMap::getClient() const { if (zookeeper_client->expired()) + { zookeeper_client = zookeeper_client->startNewSession(); + zookeeper_client->sync("/"); + } return zookeeper_client; } @@ -374,13 +402,29 @@ namespace StoragePtr create(const StorageFactory::Arguments & args) { ASTs & engine_args = args.engine_args; - if (engine_args.empty() || engine_args.size() > 1) + if (engine_args.empty() || engine_args.size() > 3) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage KeeperMap requires 1 argument: " - "keeper_path, path in the Keeper where the values will be stored"); + "Storage KeeperMap requires 1-4 arguments:\n" + "keeper_path: path in the Keeper where the values will be stored (required)\n" + "keys_limit: number of keys allowed, set to 0 for no limit (default: 0)\n" + "hosts: comma separated Keeper hosts, set to '{0}' to use the same Keeper as ClickHouse (default: '{0}')\n" + "create_missing_root_path: true if the root path should be created if it's missing (default: 1)", + default_host); - auto keeper_path = checkAndGetLiteralArgument(engine_args[0], "keeper_path"); + auto keeper_path = checkAndGetLiteralArgument(engine_args[0], "keeper_path"); + + std::string hosts = "default"; + if (engine_args.size() > 1) + hosts = checkAndGetLiteralArgument(engine_args[1], "hosts"); + + [[maybe_unused]] size_t keys_limit = 0; + if (engine_args.size() > 2) + keys_limit = checkAndGetLiteralArgument(engine_args[2], "keys_limit"); + + bool create_missing_root_path = true; + if (engine_args.size() > 3) + create_missing_root_path = checkAndGetLiteralArgument(engine_args[3], "create_missing_root_path"); StorageInMemoryMetadata metadata; metadata.setColumns(args.columns); @@ -394,7 +438,8 @@ StoragePtr create(const StorageFactory::Arguments & args) if (primary_key_names.size() != 1) throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); - return std::make_shared(args.getContext(), args.table_id, metadata, primary_key_names[0], keeper_path); + return std::make_shared( + args.getContext(), args.table_id, metadata, primary_key_names[0], keeper_path, hosts, create_missing_root_path); } } diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index bde76e958ba..6e2d6298559 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -23,7 +23,9 @@ public: const StorageID & table_id, const StorageInMemoryMetadata & metadata, std::string_view primary_key_, - std::string_view keeper_path_); + std::string_view keeper_path_, + const std::string & hosts, + bool create_missing_root_path); Pipe read( const Names & column_names, From 0651af8ac67826a6cbcfc347fa07243c8b4ff2a1 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 3 Aug 2022 14:02:14 +0000 Subject: [PATCH 04/43] Define error codes --- src/Storages/KVStorageUtils.cpp | 5 +++++ src/Storages/StorageKeeperMap.cpp | 1 + 2 files changed, 6 insertions(+) diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index 91b7e7feda6..c16ec561291 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -13,6 +13,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + namespace { // returns keys may be filter by condition diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 8e297e63273..6f0a9c01424 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -42,6 +42,7 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; extern const int KEEPER_EXCEPTION; + extern const int LOGICAL_ERROR; } namespace From bc5eda1bc785686352d015615db9904f82089cc9 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 8 Aug 2022 09:43:29 +0000 Subject: [PATCH 05/43] Key limit --- src/Storages/StorageKeeperMap.cpp | 202 ++++++++++++++++++++++++++---- src/Storages/StorageKeeperMap.h | 9 +- 2 files changed, 188 insertions(+), 23 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 6f0a9c01424..1fb0279b404 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -24,6 +24,7 @@ #include #include +#include "Common/ZooKeeper/ZooKeeper.h" #include #include #include @@ -70,6 +71,65 @@ std::string base64Decode(const std::string & encoded) constexpr std::string_view default_host = "default"; +std::string_view getBaseName(const std::string_view path) +{ + auto last_slash = path.find_last_of('/'); + if (last_slash == std::string_view::npos) + throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Failed to get basename of path '{}'", path); + + return path.substr(last_slash + 1); +} + +struct ZooKeeperLock +{ + explicit ZooKeeperLock(std::string lock_path_, zkutil::ZooKeeperPtr client_) + : lock_path(std::move(lock_path_)), client(std::move(client_)) + { + lock(); + } + + ~ZooKeeperLock() + { + if (locked) + unlock(); + } + + void lock() + { + assert(!locked); + sequence_path = client->create(std::filesystem::path(lock_path) / "lock-", "", zkutil::CreateMode::EphemeralSequential); + auto node_name = getBaseName(sequence_path); + + while (true) + { + auto children = client->getChildren(lock_path); + assert(!children.empty()); + ::sort(children.begin(), children.end()); + + auto node_it = std::find(children.begin(), children.end(), node_name); + if (node_it == children.begin()) + { + locked = true; + return; + } + + client->waitForDisappear(*(node_it - 1)); + } + } + + void unlock() + { + assert(locked); + client->remove(sequence_path); + } + +private: + std::string lock_path; + std::string sequence_path; + zkutil::ZooKeeperPtr client; + bool locked{false}; +}; + } class StorageKeeperMapSink : public SinkToStorage @@ -117,16 +177,58 @@ public: void onFinish() override { auto & zookeeper = storage.getClient(); - Coordination::Requests requests; - for (const auto & [key, value] : new_values) - { - auto path = storage.fullPathForKey(key); - if (zookeeper->exists(path)) - requests.push_back(zkutil::makeSetRequest(path, value, -1)); - else - requests.push_back(zkutil::makeCreateRequest(path, value, zkutil::CreateMode::Persistent)); + auto keys_limit = storage.keysLimit(); + + Coordination::Requests requests; + + if (!keys_limit) + { + for (const auto & [key, value] : new_values) + { + auto path = storage.fullPathForKey(key); + + if (zookeeper->exists(path)) + requests.push_back(zkutil::makeSetRequest(path, value, -1)); + else + requests.push_back(zkutil::makeCreateRequest(path, value, zkutil::CreateMode::Persistent)); + } } + else + { + ZooKeeperLock lock(storage.lockPath(), zookeeper); + + auto children = zookeeper->getChildren(storage.rootKeeperPath()); + std::unordered_set children_set(children.begin(), children.end()); + + size_t created_nodes = 0; + for (const auto & [key, value] : new_values) + { + auto path = storage.fullPathForKey(key); + + if (children_set.contains(key)) + { + requests.push_back(zkutil::makeSetRequest(path, value, -1)); + } + else + { + requests.push_back(zkutil::makeCreateRequest(path, value, zkutil::CreateMode::Persistent)); + ++created_nodes; + } + } + + size_t keys_num_after_insert = children.size() - 1 + created_nodes; + if (keys_limit && keys_num_after_insert > keys_limit) + { + throw Exception( + DB::ErrorCodes::BAD_ARGUMENTS, + "Cannot insert values. {} key would be created setting the total keys number to {} exceeding the limit of {}", + created_nodes, + keys_num_after_insert, + keys_limit); + } + } + zookeeper->multi(requests); } @@ -191,13 +293,13 @@ public: namespace { -zkutil::ZooKeeperPtr getZooKeeperClient(const std::string & hosts, const ContextPtr & context) -{ - if (hosts == default_host) - return context->getZooKeeper()->startNewSession(); + zkutil::ZooKeeperPtr getZooKeeperClient(const std::string & hosts, const ContextPtr & context) + { + if (hosts == default_host) + return context->getZooKeeper()->startNewSession(); - return std::make_shared(hosts); -} + return std::make_shared(hosts); + } } @@ -208,7 +310,8 @@ StorageKeeperMap::StorageKeeperMap( std::string_view primary_key_, std::string_view keeper_path_, const std::string & hosts, - bool create_missing_root_path) + bool create_missing_root_path, + size_t keys_limit_) : IKeyValueStorage(table_id), keeper_path(keeper_path_), primary_key(primary_key_), zookeeper_client(getZooKeeperClient(hosts, context)) { setInMemoryMetadata(metadata); @@ -219,6 +322,7 @@ StorageKeeperMap::StorageKeeperMap( throw Exception("keeper_path should start with '/'", ErrorCodes::BAD_ARGUMENTS); auto client = getClient(); + if (keeper_path != "/" && !client->exists(keeper_path)) { if (!create_missing_root_path) @@ -241,12 +345,46 @@ StorageKeeperMap::StorageKeeperMap( throw Exception("keeper_path is invalid, contains subsequent '/'", ErrorCodes::BAD_ARGUMENTS); auto path = keeper_path.substr(0, cur_pos); - auto status = client->tryCreate(path, "", zkutil::CreateMode::Persistent); - if (status != Coordination::Error::ZOK && status != Coordination::Error::ZNODEEXISTS) - throw zkutil::KeeperException(status, path); + client->createIfNotExists(path, ""); } while (cur_pos != std::string_view::npos); } } + + // create metadata nodes + std::filesystem::path root_path{keeper_path}; + + auto metadata_path_fs = root_path / "__ch_metadata"; + metadata_path = metadata_path_fs; + client->createIfNotExists(metadata_path, ""); + + lock_path = metadata_path_fs / "lock"; + client->createIfNotExists(lock_path, ""); + + auto keys_limit_path = metadata_path_fs / "keys_limit"; + auto status = client->tryCreate(keys_limit_path, toString(keys_limit_), zkutil::CreateMode::Persistent); + if (status == Coordination::Error::ZNODEEXISTS) + { + auto data = client->get(keys_limit_path, nullptr, nullptr); + UInt64 stored_keys_limit = parse(data); + if (stored_keys_limit != keys_limit_) + { + keys_limit = stored_keys_limit; + LOG_WARNING( + &Poco::Logger::get("StorageKeeperMap"), + "Keys limit is already set for {} to {}. Going to use already set value", + keeper_path, + stored_keys_limit); + } + } + else if (status == Coordination::Error::ZOK) + { + keys_limit = keys_limit_; + } + else + { + throw zkutil::KeeperException(status, keys_limit_path); + } + LOG_INFO(&Poco::Logger::get("LOGGER"), "Keys limit set to {}", keys_limit); } @@ -254,7 +392,7 @@ Pipe StorageKeeperMap::read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, - ContextPtr /*context*/, + ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, unsigned num_streams) @@ -266,7 +404,7 @@ Pipe StorageKeeperMap::read( Block sample_block = storage_snapshot->metadata->getSampleBlock(); auto primary_key_type = sample_block.getByName(primary_key).type; - std::tie(filtered_keys, all_scan) = getFilterKeys(primary_key, primary_key_type, query_info); + std::tie(filtered_keys, all_scan) = getFilterKeys(primary_key, primary_key_type, query_info, context); const auto process_keys = [&](KeyContainerPtr keys) -> Pipe { @@ -329,6 +467,16 @@ std::string StorageKeeperMap::fullPathForKey(const std::string_view key) const return fmt::format("{}/{}", keeper_path, key); } +const std::string & StorageKeeperMap::lockPath() const +{ + return lock_path; +} + +UInt64 StorageKeeperMap::keysLimit() const +{ + return keys_limit; +} + Chunk StorageKeeperMap::getByKeys(const ColumnsWithTypeAndName & keys, PaddedPODArray & null_map) const { if (keys.size() != 1) @@ -361,7 +509,14 @@ Chunk StorageKeeperMap::getBySerializedKeys(const std::span k for (const auto & key : keys) { - values.emplace_back(client->asyncTryGet(fullPathForKey(key))); + const auto full_path = fullPathForKey(key); + if (full_path == metadata_path) + { + values.emplace_back(); + continue; + } + + values.emplace_back(client->asyncTryGet(full_path)); } auto wait_until = std::chrono::system_clock::now() + std::chrono::milliseconds(Coordination::DEFAULT_OPERATION_TIMEOUT_MS); @@ -369,6 +524,9 @@ Chunk StorageKeeperMap::getBySerializedKeys(const std::span k for (size_t i = 0; i < keys.size(); ++i) { auto & value = values[i]; + if (!value.valid()) + continue; + if (value.wait_until(wait_until) != std::future_status::ready) throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Failed to fetch values: timeout"); @@ -440,7 +598,7 @@ StoragePtr create(const StorageFactory::Arguments & args) throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); return std::make_shared( - args.getContext(), args.table_id, metadata, primary_key_names[0], keeper_path, hosts, create_missing_root_path); + args.getContext(), args.table_id, metadata, primary_key_names[0], keeper_path, hosts, create_missing_root_path, keys_limit); } } diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 6e2d6298559..4a739311f2a 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -25,7 +25,8 @@ public: std::string_view primary_key_, std::string_view keeper_path_, const std::string & hosts, - bool create_missing_root_path); + bool create_missing_root_path, + size_t keys_limit); Pipe read( const Names & column_names, @@ -56,9 +57,15 @@ public: const std::string & rootKeeperPath() const; std::string fullPathForKey(std::string_view key) const; + const std::string & lockPath() const; + UInt64 keysLimit() const; + private: std::string keeper_path; std::string primary_key; + std::string metadata_path; + std::string lock_path; + UInt64 keys_limit{0}; mutable zkutil::ZooKeeperPtr zookeeper_client; }; From 617ef00908f4362e9afd7ebcf072b2d27e3350b5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 8 Aug 2022 14:09:37 +0000 Subject: [PATCH 06/43] Small fixes --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Storages/KVStorageUtils.h | 2 ++ src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 1 - src/Storages/StorageKeeperMap.cpp | 19 +++++++++---------- src/Storages/StorageKeeperMap.h | 2 +- 5 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ea11a1b23ac..7a00bbf524c 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -720,7 +720,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti } /// We can have queries like "CREATE TABLE
ENGINE=" if /// supports schema inference (will determine table structure in it's constructor). - else if (!StorageFactory::instance().checkIfStorageSupportsSchemaInterface(create.storage->engine->name) && create.storage->engine->name != "KeeperMap") // NOLINT + else if (!StorageFactory::instance().checkIfStorageSupportsSchemaInterface(create.storage->engine->name)) // NOLINT throw Exception("Incorrect CREATE query: required list of column descriptions or AS section or SELECT.", ErrorCodes::INCORRECT_QUERY); /// Even if query has list of columns, canonicalize it (unfold Nested columns). diff --git a/src/Storages/KVStorageUtils.h b/src/Storages/KVStorageUtils.h index ca6aecd1374..3807e5f084b 100644 --- a/src/Storages/KVStorageUtils.h +++ b/src/Storages/KVStorageUtils.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -29,6 +30,7 @@ void fillColumns(const K & key, const V & value, size_t key_pos, const Block & h for (size_t i = 0; i < header.columns(); ++i) { const auto & serialization = header.getByPosition(i).type->getDefaultSerialization(); + LOG_INFO(&Poco::Logger::get("LOGGER"), "Reading coluimn {} of type {}", i, columns[i]->getDataType()); serialization->deserializeBinary(*columns[i], i == key_pos ? key_buffer : value_buffer); } } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 169b6c1e1bc..62c9a0eeae7 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 1fb0279b404..081861653ae 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -24,7 +24,7 @@ #include #include -#include "Common/ZooKeeper/ZooKeeper.h" +#include #include #include #include @@ -293,13 +293,13 @@ public: namespace { - zkutil::ZooKeeperPtr getZooKeeperClient(const std::string & hosts, const ContextPtr & context) - { - if (hosts == default_host) - return context->getZooKeeper()->startNewSession(); +zkutil::ZooKeeperPtr getZooKeeperClient(const std::string & hosts, const ContextPtr & context) +{ + if (hosts == default_host) + return context->getZooKeeper()->startNewSession(); - return std::make_shared(hosts); - } + return std::make_shared(hosts); +} } @@ -384,7 +384,6 @@ StorageKeeperMap::StorageKeeperMap( { throw zkutil::KeeperException(status, keys_limit_path); } - LOG_INFO(&Poco::Logger::get("LOGGER"), "Keys limit set to {}", keys_limit); } @@ -561,7 +560,7 @@ namespace StoragePtr create(const StorageFactory::Arguments & args) { ASTs & engine_args = args.engine_args; - if (engine_args.empty() || engine_args.size() > 3) + if (engine_args.empty() || engine_args.size() > 4) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage KeeperMap requires 1-4 arguments:\n" @@ -577,7 +576,7 @@ StoragePtr create(const StorageFactory::Arguments & args) if (engine_args.size() > 1) hosts = checkAndGetLiteralArgument(engine_args[1], "hosts"); - [[maybe_unused]] size_t keys_limit = 0; + size_t keys_limit = 0; if (engine_args.size() > 2) keys_limit = checkAndGetLiteralArgument(engine_args[2], "keys_limit"); diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 4a739311f2a..fdbda1cde34 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -5,7 +5,7 @@ #include #include #include -#include "Common/PODArray_fwd.h" +#include #include #include From 4023d4a37af0f739f88b7eec3d7e691cb990b1e8 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 10 Aug 2022 07:24:56 +0000 Subject: [PATCH 07/43] Add support for drop/truncate and tests --- src/Common/ZooKeeper/ZooKeeper.cpp | 12 +-- src/Common/ZooKeeper/ZooKeeper.h | 2 +- src/Storages/KVStorageUtils.h | 1 - src/Storages/StorageKeeperMap.cpp | 86 ++++++++++++------- src/Storages/StorageKeeperMap.h | 8 +- .../0_stateless/02381_keeper_map.reference | 6 ++ .../queries/0_stateless/02381_keeper_map.sql | 42 +++++++++ 7 files changed, 114 insertions(+), 43 deletions(-) create mode 100644 tests/queries/0_stateless/02381_keeper_map.reference create mode 100644 tests/queries/0_stateless/02381_keeper_map.sql diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 96abf3b543a..74945e3b50c 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -737,7 +737,7 @@ void ZooKeeper::removeChildrenRecursive(const std::string & path, const String & } } -bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probably_flat, const String & keep_child_node) +bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probably_flat, const std::string_view keep_child_node) { Strings children; if (tryGetChildren(path, children) != Coordination::Error::ZOK) @@ -754,13 +754,13 @@ bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probab { String child_path = fs::path(path) / children.back(); - /// Will try to avoid recursive getChildren calls if child_path probably has no children. - /// It may be extremely slow when path contain a lot of leaf children. - if (!probably_flat) - tryRemoveChildrenRecursive(child_path); - if (likely(keep_child_node.empty() || keep_child_node != children.back())) { + /// Will try to avoid recursive getChildren calls if child_path probably has no children. + /// It may be extremely slow when path contain a lot of leaf children. + if (!probably_flat) + tryRemoveChildrenRecursive(child_path); + batch.push_back(child_path); ops.emplace_back(zkutil::makeRemoveRequest(child_path, -1)); } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index c9b5dc69499..416fc78d814 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -238,7 +238,7 @@ public: /// If probably_flat is true, this method will optimistically try to remove children non-recursive /// and will fall back to recursive removal if it gets ZNOTEMPTY for some child. /// Returns true if no kind of fallback happened. - bool tryRemoveChildrenRecursive(const std::string & path, bool probably_flat = false, const String & keep_child_node = {}); + bool tryRemoveChildrenRecursive(const std::string & path, bool probably_flat = false, std::string_view keep_child_node = {}); /// Remove all children nodes (non recursive). void removeChildren(const std::string & path); diff --git a/src/Storages/KVStorageUtils.h b/src/Storages/KVStorageUtils.h index 3807e5f084b..e3216164869 100644 --- a/src/Storages/KVStorageUtils.h +++ b/src/Storages/KVStorageUtils.h @@ -30,7 +30,6 @@ void fillColumns(const K & key, const V & value, size_t key_pos, const Block & h for (size_t i = 0; i < header.columns(); ++i) { const auto & serialization = header.getByPosition(i).type->getDefaultSerialization(); - LOG_INFO(&Poco::Logger::get("LOGGER"), "Reading coluimn {} of type {}", i, columns[i]->getDataType()); serialization->deserializeBinary(*columns[i], i == key_pos ? key_buffer : value_buffer); } } diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 081861653ae..3096ea0ba72 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -82,10 +82,11 @@ std::string_view getBaseName(const std::string_view path) struct ZooKeeperLock { - explicit ZooKeeperLock(std::string lock_path_, zkutil::ZooKeeperPtr client_) + explicit ZooKeeperLock(std::string lock_path_, zkutil::ZooKeeperPtr client_, bool defer_lock = false) : lock_path(std::move(lock_path_)), client(std::move(client_)) { - lock(); + if (!defer_lock) + lock(); } ~ZooKeeperLock() @@ -120,7 +121,7 @@ struct ZooKeeperLock void unlock() { assert(locked); - client->remove(sequence_path); + client->tryRemove(sequence_path); } private: @@ -307,11 +308,13 @@ StorageKeeperMap::StorageKeeperMap( ContextPtr context, const StorageID & table_id, const StorageInMemoryMetadata & metadata, + bool attach, std::string_view primary_key_, std::string_view keeper_path_, const std::string & hosts, bool create_missing_root_path, - size_t keys_limit_) + size_t keys_limit_, + bool remove_existing_data) : IKeyValueStorage(table_id), keeper_path(keeper_path_), primary_key(primary_key_), zookeeper_client(getZooKeeperClient(hosts, context)) { setInMemoryMetadata(metadata); @@ -323,6 +326,12 @@ StorageKeeperMap::StorageKeeperMap( auto client = getClient(); + if (attach) + { + // validate all metadata nodes are present + return; + } + if (keeper_path != "/" && !client->exists(keeper_path)) { if (!create_missing_root_path) @@ -350,40 +359,32 @@ StorageKeeperMap::StorageKeeperMap( } } + Coordination::Stat stats; + auto exists = client->exists(keeper_path, &stats); + if (!exists) + throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Path '{}' should exist, but was deleted from another source", keeper_path); + + if (stats.numChildren != 0) + { + if (!remove_existing_data) + throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Cannot create StorageKeeperMap using '{}' as path because it contains children nodes", keeper_path); + + LOG_INFO(&Poco::Logger::get("StorageKeepermap"), "Removing children for {} because remove_existing_data was set to true.", keeper_path); + client->removeChildrenRecursive(keeper_path); + } + // create metadata nodes std::filesystem::path root_path{keeper_path}; auto metadata_path_fs = root_path / "__ch_metadata"; metadata_path = metadata_path_fs; - client->createIfNotExists(metadata_path, ""); + client->create(metadata_path, "", zkutil::CreateMode::Persistent); lock_path = metadata_path_fs / "lock"; - client->createIfNotExists(lock_path, ""); + client->create(lock_path, "", zkutil::CreateMode::Persistent); auto keys_limit_path = metadata_path_fs / "keys_limit"; - auto status = client->tryCreate(keys_limit_path, toString(keys_limit_), zkutil::CreateMode::Persistent); - if (status == Coordination::Error::ZNODEEXISTS) - { - auto data = client->get(keys_limit_path, nullptr, nullptr); - UInt64 stored_keys_limit = parse(data); - if (stored_keys_limit != keys_limit_) - { - keys_limit = stored_keys_limit; - LOG_WARNING( - &Poco::Logger::get("StorageKeeperMap"), - "Keys limit is already set for {} to {}. Going to use already set value", - keeper_path, - stored_keys_limit); - } - } - else if (status == Coordination::Error::ZOK) - { - keys_limit = keys_limit_; - } - else - { - throw zkutil::KeeperException(status, keys_limit_path); - } + client->create(keys_limit_path, toString(keys_limit_), zkutil::CreateMode::Persistent); } @@ -445,6 +446,20 @@ SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const Storage return std::make_shared(*this, metadata_snapshot); } +void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr & , ContextPtr, TableExclusiveLockHolder &) +{ + auto client = getClient(); + + ZooKeeperLock keeper_lock(lockPath(), client); + client->tryRemoveChildrenRecursive(keeper_path, /*probably_flat*/ true, getBaseName(metadata_path)); +} + +void StorageKeeperMap::drop() +{ + auto client = getClient(); + client->tryRemoveChildrenRecursive(keeper_path, /*probably_flat*/ false); +} + zkutil::ZooKeeperPtr & StorageKeeperMap::getClient() const { if (zookeeper_client->expired()) @@ -563,11 +578,12 @@ StoragePtr create(const StorageFactory::Arguments & args) if (engine_args.empty() || engine_args.size() > 4) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage KeeperMap requires 1-4 arguments:\n" + "Storage KeeperMap requires 1-5 arguments:\n" "keeper_path: path in the Keeper where the values will be stored (required)\n" - "keys_limit: number of keys allowed, set to 0 for no limit (default: 0)\n" "hosts: comma separated Keeper hosts, set to '{0}' to use the same Keeper as ClickHouse (default: '{0}')\n" - "create_missing_root_path: true if the root path should be created if it's missing (default: 1)", + "keys_limit: number of keys allowed, set to 0 for no limit (default: 0)\n" + "create_missing_root_path: 1 if the root path should be created if it's missing, otherwise throw exception (default: 1)\n", + "remove_existing_data: true if children inside 'keeper_path' should be deleted, otherwise throw exception (default: 0)", default_host); auto keeper_path = checkAndGetLiteralArgument(engine_args[0], "keeper_path"); @@ -584,6 +600,10 @@ StoragePtr create(const StorageFactory::Arguments & args) if (engine_args.size() > 3) create_missing_root_path = checkAndGetLiteralArgument(engine_args[3], "create_missing_root_path"); + bool remove_existing_data = false; + if (engine_args.size() > 4) + create_missing_root_path = checkAndGetLiteralArgument(engine_args[4], "create_missing_root_path"); + StorageInMemoryMetadata metadata; metadata.setColumns(args.columns); metadata.setConstraints(args.constraints); @@ -597,7 +617,7 @@ StoragePtr create(const StorageFactory::Arguments & args) throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); return std::make_shared( - args.getContext(), args.table_id, metadata, primary_key_names[0], keeper_path, hosts, create_missing_root_path, keys_limit); + args.getContext(), args.table_id, metadata, args.query.attach, primary_key_names[0], keeper_path, hosts, create_missing_root_path, keys_limit, remove_existing_data); } } diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index fdbda1cde34..e1ddc304c68 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -17,16 +17,17 @@ namespace DB class StorageKeeperMap final : public IKeyValueStorage { public: - // TODO(antonio2368): add setting to control creating if keeper_path doesn't exist StorageKeeperMap( ContextPtr context, const StorageID & table_id, const StorageInMemoryMetadata & metadata, + bool attach, std::string_view primary_key_, std::string_view keeper_path_, const std::string & hosts, bool create_missing_root_path, - size_t keys_limit); + size_t keys_limit, + bool remove_existing_data); Pipe read( const Names & column_names, @@ -39,6 +40,9 @@ public: SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; + void truncate(const ASTPtr &, const StorageMetadataPtr & , ContextPtr, TableExclusiveLockHolder &) override; + void drop() override; + std::string getName() const override { return "KeeperMap"; } Names getPrimaryKey() const override { return {primary_key}; } diff --git a/tests/queries/0_stateless/02381_keeper_map.reference b/tests/queries/0_stateless/02381_keeper_map.reference new file mode 100644 index 00000000000..eea8dd975e8 --- /dev/null +++ b/tests/queries/0_stateless/02381_keeper_map.reference @@ -0,0 +1,6 @@ +1 +1 +1 +1 +1 1 1 1 1 +1 diff --git a/tests/queries/0_stateless/02381_keeper_map.sql b/tests/queries/0_stateless/02381_keeper_map.sql new file mode 100644 index 00000000000..24048a67cfd --- /dev/null +++ b/tests/queries/0_stateless/02381_keeper_map.sql @@ -0,0 +1,42 @@ +-- Tags: no-ordinary-database, no-fasttest +-- Tag no-ordinary-database: Sometimes cannot lock file most likely due to concurrent or adjacent tests, but we don't care how it works in Ordinary database +-- Tag no-fasttest: In fasttest, ENABLE_LIBRARIES=0, so rocksdb engine is not enabled by default + +SET database_atomic_wait_for_drop_and_detach_synchronously = 1; + +DROP TABLE IF EXISTS 02381_test; + +CREATE TABLE 02381_test (key String, value UInt32) Engine=KeeperMap('/test2381'); -- { serverError 36 } +CREATE TABLE 02381_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key2); -- { serverError 47 } +CREATE TABLE 02381_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key, value); -- { serverError 36 } +CREATE TABLE 02381_test (key Tuple(String, UInt32), value UInt64) Engine=KeeperMap('/test2381') PRIMARY KEY(key); + +DROP TABLE IF EXISTS 02381_test; +CREATE TABLE 02381_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key); + +INSERT INTO 02381_test SELECT '1_1', number FROM numbers(10000); +SELECT COUNT(1) == 1 FROM 02381_test; + +INSERT INTO 02381_test SELECT concat(toString(number), '_1'), number FROM numbers(10000); +SELECT COUNT(1) == 10000 FROM 02381_test; +SELECT uniqExact(key) == 32 FROM (SELECT * FROM 02381_test LIMIT 32 SETTINGS max_block_size = 1); +SELECT SUM(value) == 1 + 99 + 900 FROM 02381_test WHERE key IN ('1_1', '99_1', '900_1'); + +DROP TABLE IF EXISTS 02381_test; +DROP TABLE IF EXISTS 02381_test_memory; + +CREATE TABLE 02381_test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=KeeperMap('/test2381') PRIMARY KEY(k); +CREATE TABLE 02381_test_memory AS 02381_test Engine = Memory; + +INSERT INTO 02381_test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; + +INSERT INTO 02381_test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; + + +SELECT A.a = B.a, A.b = B.b, A.c = B.c, A.d = B.d, A.e = B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 02381_test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 02381_test_memory) B USING a ORDER BY a; + +TRUNCATE TABLE 02381_test; +SELECT 0 == COUNT(1) FROM 02381_test; + +DROP TABLE IF EXISTS 02381_test; +DROP TABLE IF EXISTS 02381_test_memory; From e19ecd95d6ac8b4f5cdf9cc0710c59b6140aced1 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 10 Aug 2022 08:52:36 +0000 Subject: [PATCH 08/43] Add basic support for attach --- src/Storages/StorageKeeperMap.cpp | 83 +++++++++++++++++-------------- src/Storages/StorageKeeperMap.h | 6 ++- 2 files changed, 50 insertions(+), 39 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 3096ea0ba72..f90e6d4e366 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -310,81 +310,90 @@ StorageKeeperMap::StorageKeeperMap( const StorageInMemoryMetadata & metadata, bool attach, std::string_view primary_key_, - std::string_view keeper_path_, + std::string_view root_path_, const std::string & hosts, bool create_missing_root_path, size_t keys_limit_, bool remove_existing_data) - : IKeyValueStorage(table_id), keeper_path(keeper_path_), primary_key(primary_key_), zookeeper_client(getZooKeeperClient(hosts, context)) + : IKeyValueStorage(table_id), root_path(root_path_), primary_key(primary_key_), zookeeper_client(getZooKeeperClient(hosts, context)), log(&Poco::Logger::get("StorageKeeperMap")) { setInMemoryMetadata(metadata); - if (keeper_path.empty()) - throw Exception("keeper_path should not be empty", ErrorCodes::BAD_ARGUMENTS); - if (!keeper_path.starts_with('/')) - throw Exception("keeper_path should start with '/'", ErrorCodes::BAD_ARGUMENTS); + if (root_path.empty()) + throw Exception("root_path should not be empty", ErrorCodes::BAD_ARGUMENTS); + if (!root_path.starts_with('/')) + throw Exception("root_path should start with '/'", ErrorCodes::BAD_ARGUMENTS); auto client = getClient(); + std::filesystem::path root_path_fs{root_path}; + auto metadata_path_fs = root_path_fs / "__ch_metadata"; + metadata_path = metadata_path_fs; + lock_path = metadata_path_fs / "lock"; + auto keys_limit_path = metadata_path_fs / "keys_limit"; + if (attach) { // validate all metadata nodes are present + Coordination::Requests requests; + + requests.push_back(zkutil::makeCheckRequest(root_path, -1)); + requests.push_back(zkutil::makeCheckRequest(metadata_path, -1)); + requests.push_back(zkutil::makeCheckRequest(lock_path, -1)); + requests.push_back(zkutil::makeCheckRequest(keys_limit_path, -1)); + + client->multi(requests); return; } - if (keeper_path != "/" && !client->exists(keeper_path)) + if (root_path != "/" && !client->exists(root_path)) { if (!create_missing_root_path) { throw Exception( ErrorCodes::BAD_ARGUMENTS, "Path '{}' doesn't exist. Please create it or set 'create_missing_root_path' to true'", - keeper_path_); + root_path_); } else { - LOG_TRACE(&Poco::Logger::get("StorageKeeperMap"), "Creating root path {}", keeper_path); + LOG_TRACE(log, "Creating root path {}", root_path); size_t cur_pos = 0; do { size_t search_start = cur_pos + 1; - cur_pos = keeper_path.find('/', search_start); + cur_pos = root_path.find('/', search_start); if (search_start == cur_pos) - throw Exception("keeper_path is invalid, contains subsequent '/'", ErrorCodes::BAD_ARGUMENTS); + throw Exception("root_path is invalid, contains subsequent '/'", ErrorCodes::BAD_ARGUMENTS); - auto path = keeper_path.substr(0, cur_pos); + auto path = root_path.substr(0, cur_pos); client->createIfNotExists(path, ""); } while (cur_pos != std::string_view::npos); } } Coordination::Stat stats; - auto exists = client->exists(keeper_path, &stats); + auto exists = client->exists(root_path, &stats); if (!exists) - throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Path '{}' should exist, but was deleted from another source", keeper_path); + throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Path '{}' should exist, but was deleted from another source", root_path); if (stats.numChildren != 0) { if (!remove_existing_data) - throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Cannot create StorageKeeperMap using '{}' as path because it contains children nodes", keeper_path); + throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Cannot create StorageKeeperMap using '{}' as path because it contains children nodes. Set remove_existing_data to 1 to clear children nodes", root_path); - LOG_INFO(&Poco::Logger::get("StorageKeepermap"), "Removing children for {} because remove_existing_data was set to true.", keeper_path); - client->removeChildrenRecursive(keeper_path); + LOG_INFO(log, "Removing children for {} because remove_existing_data was set to true.", root_path); + client->removeChildrenRecursive(root_path); } // create metadata nodes - std::filesystem::path root_path{keeper_path}; + Coordination::Requests create_requests; + create_requests.push_back(zkutil::makeCreateRequest(metadata_path, "", zkutil::CreateMode::Persistent)); + create_requests.push_back(zkutil::makeCreateRequest(lock_path, "", zkutil::CreateMode::Persistent)); + create_requests.push_back(zkutil::makeCreateRequest(keys_limit_path, toString(keys_limit_), zkutil::CreateMode::Persistent)); - auto metadata_path_fs = root_path / "__ch_metadata"; - metadata_path = metadata_path_fs; - client->create(metadata_path, "", zkutil::CreateMode::Persistent); - - lock_path = metadata_path_fs / "lock"; - client->create(lock_path, "", zkutil::CreateMode::Persistent); - - auto keys_limit_path = metadata_path_fs / "keys_limit"; - client->create(keys_limit_path, toString(keys_limit_), zkutil::CreateMode::Persistent); + client->multi(create_requests); } @@ -436,7 +445,7 @@ Pipe StorageKeeperMap::read( auto & client = getClient(); if (all_scan) - return process_keys(std::make_shared>(client->getChildren(keeper_path))); + return process_keys(std::make_shared>(client->getChildren(root_path))); return process_keys(std::move(filtered_keys)); } @@ -451,13 +460,13 @@ void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr & , Con auto client = getClient(); ZooKeeperLock keeper_lock(lockPath(), client); - client->tryRemoveChildrenRecursive(keeper_path, /*probably_flat*/ true, getBaseName(metadata_path)); + client->tryRemoveChildrenRecursive(root_path, /*probably_flat*/ true, getBaseName(metadata_path)); } void StorageKeeperMap::drop() { auto client = getClient(); - client->tryRemoveChildrenRecursive(keeper_path, /*probably_flat*/ false); + client->tryRemoveChildrenRecursive(root_path, /*probably_flat*/ false); } zkutil::ZooKeeperPtr & StorageKeeperMap::getClient() const @@ -473,12 +482,12 @@ zkutil::ZooKeeperPtr & StorageKeeperMap::getClient() const const std::string & StorageKeeperMap::rootKeeperPath() const { - return keeper_path; + return root_path; } std::string StorageKeeperMap::fullPathForKey(const std::string_view key) const { - return fmt::format("{}/{}", keeper_path, key); + return fmt::format("{}/{}", root_path, key); } const std::string & StorageKeeperMap::lockPath() const @@ -579,14 +588,14 @@ StoragePtr create(const StorageFactory::Arguments & args) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage KeeperMap requires 1-5 arguments:\n" - "keeper_path: path in the Keeper where the values will be stored (required)\n" + "root_path: path in the Keeper where the values will be stored (required)\n" "hosts: comma separated Keeper hosts, set to '{0}' to use the same Keeper as ClickHouse (default: '{0}')\n" "keys_limit: number of keys allowed, set to 0 for no limit (default: 0)\n" "create_missing_root_path: 1 if the root path should be created if it's missing, otherwise throw exception (default: 1)\n", - "remove_existing_data: true if children inside 'keeper_path' should be deleted, otherwise throw exception (default: 0)", + "remove_existing_data: true if children inside 'root_path' should be deleted, otherwise throw exception (default: 0)", default_host); - auto keeper_path = checkAndGetLiteralArgument(engine_args[0], "keeper_path"); + auto root_path = checkAndGetLiteralArgument(engine_args[0], "root_path"); std::string hosts = "default"; if (engine_args.size() > 1) @@ -602,7 +611,7 @@ StoragePtr create(const StorageFactory::Arguments & args) bool remove_existing_data = false; if (engine_args.size() > 4) - create_missing_root_path = checkAndGetLiteralArgument(engine_args[4], "create_missing_root_path"); + remove_existing_data = checkAndGetLiteralArgument(engine_args[4], "remove_existing_data"); StorageInMemoryMetadata metadata; metadata.setColumns(args.columns); @@ -617,7 +626,7 @@ StoragePtr create(const StorageFactory::Arguments & args) throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); return std::make_shared( - args.getContext(), args.table_id, metadata, args.query.attach, primary_key_names[0], keeper_path, hosts, create_missing_root_path, keys_limit, remove_existing_data); + args.getContext(), args.table_id, metadata, args.query.attach, primary_key_names[0], root_path, hosts, create_missing_root_path, keys_limit, remove_existing_data); } } diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index e1ddc304c68..fd243206721 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -23,7 +23,7 @@ public: const StorageInMemoryMetadata & metadata, bool attach, std::string_view primary_key_, - std::string_view keeper_path_, + std::string_view root_path_, const std::string & hosts, bool create_missing_root_path, size_t keys_limit, @@ -65,13 +65,15 @@ public: UInt64 keysLimit() const; private: - std::string keeper_path; + std::string root_path; std::string primary_key; std::string metadata_path; std::string lock_path; UInt64 keys_limit{0}; mutable zkutil::ZooKeeperPtr zookeeper_client; + + Poco::Logger * log; }; } From 18790539c8f162751be779a66aa1320b02a345d1 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 10 Aug 2022 08:52:47 +0000 Subject: [PATCH 09/43] Update test --- tests/queries/0_stateless/02381_keeper_map.sql | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02381_keeper_map.sql b/tests/queries/0_stateless/02381_keeper_map.sql index 24048a67cfd..9a4f0b367f7 100644 --- a/tests/queries/0_stateless/02381_keeper_map.sql +++ b/tests/queries/0_stateless/02381_keeper_map.sql @@ -1,6 +1,4 @@ --- Tags: no-ordinary-database, no-fasttest --- Tag no-ordinary-database: Sometimes cannot lock file most likely due to concurrent or adjacent tests, but we don't care how it works in Ordinary database --- Tag no-fasttest: In fasttest, ENABLE_LIBRARIES=0, so rocksdb engine is not enabled by default +-- Tags: no-fasttest SET database_atomic_wait_for_drop_and_detach_synchronously = 1; From fc33941959e552edc4c2081d08a2e86422bbd85b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 23 Aug 2022 13:15:31 +0000 Subject: [PATCH 10/43] Add support for shared drop/create --- src/Common/ErrorCodes.cpp | 1 + src/Common/ZooKeeper/ZooKeeper.cpp | 16 +- src/Common/ZooKeeper/ZooKeeper.h | 5 +- src/Storages/StorageKeeperMap.cpp | 340 ++++++++++-------- src/Storages/StorageKeeperMap.h | 62 +++- .../queries/0_stateless/02381_keeper_map.sql | 1 - 6 files changed, 255 insertions(+), 170 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index f8d8deab08b..a30d6814516 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -635,6 +635,7 @@ M(664, ACCESS_STORAGE_DOESNT_ALLOW_BACKUP) \ M(665, CANNOT_CONNECT_NATS) \ M(666, CANNOT_USE_CACHE) \ + M(667, INVALID_STATE) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 74945e3b50c..b47ab35820e 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -720,7 +720,7 @@ void ZooKeeper::removeChildren(const std::string & path) } -void ZooKeeper::removeChildrenRecursive(const std::string & path, const String & keep_child_node) +void ZooKeeper::removeChildrenRecursive(const std::string & path, const std::string_view keep_child_node) { Strings children = getChildren(path); while (!children.empty()) @@ -728,9 +728,11 @@ void ZooKeeper::removeChildrenRecursive(const std::string & path, const String & Coordination::Requests ops; for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i) { - removeChildrenRecursive(fs::path(path) / children.back()); if (likely(keep_child_node.empty() || keep_child_node != children.back())) + { + removeChildrenRecursive(fs::path(path) / children.back()); ops.emplace_back(makeRemoveRequest(fs::path(path) / children.back(), -1)); + } children.pop_back(); } multi(ops); @@ -754,13 +756,13 @@ bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probab { String child_path = fs::path(path) / children.back(); + /// Will try to avoid recursive getChildren calls if child_path probably has no children. + /// It may be extremely slow when path contain a lot of leaf children. + if (!probably_flat) + tryRemoveChildrenRecursive(child_path); + if (likely(keep_child_node.empty() || keep_child_node != children.back())) { - /// Will try to avoid recursive getChildren calls if child_path probably has no children. - /// It may be extremely slow when path contain a lot of leaf children. - if (!probably_flat) - tryRemoveChildrenRecursive(child_path); - batch.push_back(child_path); ops.emplace_back(zkutil::makeRemoveRequest(child_path, -1)); } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 416fc78d814..d16f1f96271 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -232,9 +232,8 @@ public: void tryRemoveRecursive(const std::string & path); /// Similar to removeRecursive(...) and tryRemoveRecursive(...), but does not remove path itself. - /// If keep_child_node is not empty, this method will not remove path/keep_child_node (but will remove its subtree). - /// It can be useful to keep some child node as a flag which indicates that path is currently removing. - void removeChildrenRecursive(const std::string & path, const String & keep_child_node = {}); + /// If keep_child_node is not empty, this method will not remove path/keep_child_node and its subtree. + void removeChildrenRecursive(const std::string & path, std::string_view keep_child_node = {}); /// If probably_flat is true, this method will optimistically try to remove children non-recursive /// and will fall back to recursive removal if it gets ZNOTEMPTY for some child. /// Returns true if no kind of fallback happened. diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 962d008882e..3477f29fc10 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -23,9 +23,10 @@ #include #include -#include +#include "Common/Exception.h" #include #include +#include #include #include @@ -123,6 +124,15 @@ struct ZooKeeperLock client->tryRemove(sequence_path); } + // release the ownership and return the path + // for the lock + std::string release() + { + assert(locked); + locked = false; + return std::move(sequence_path); + } + private: std::string lock_path; std::string sequence_path; @@ -178,57 +188,17 @@ public: { auto & zookeeper = storage.getClient(); - auto keys_limit = storage.keysLimit(); - Coordination::Requests requests; - if (!keys_limit) + for (const auto & [key, value] : new_values) { - for (const auto & [key, value] : new_values) - { - auto path = storage.fullPathForKey(key); + auto path = storage.fullPathForKey(key); - if (zookeeper->exists(path)) - requests.push_back(zkutil::makeSetRequest(path, value, -1)); - else - requests.push_back(zkutil::makeCreateRequest(path, value, zkutil::CreateMode::Persistent)); - } + if (zookeeper->exists(path)) + requests.push_back(zkutil::makeSetRequest(path, value, -1)); + else + requests.push_back(zkutil::makeCreateRequest(path, value, zkutil::CreateMode::Persistent)); } - else - { - ZooKeeperLock lock(storage.lockPath(), zookeeper); - - auto children = zookeeper->getChildren(storage.rootKeeperPath()); - std::unordered_set children_set(children.begin(), children.end()); - - size_t created_nodes = 0; - for (const auto & [key, value] : new_values) - { - auto path = storage.fullPathForKey(key); - - if (children_set.contains(key)) - { - requests.push_back(zkutil::makeSetRequest(path, value, -1)); - } - else - { - requests.push_back(zkutil::makeCreateRequest(path, value, zkutil::CreateMode::Persistent)); - ++created_nodes; - } - } - - size_t keys_num_after_insert = children.size() - 1 + created_nodes; - if (keys_limit && keys_num_after_insert > keys_limit) - { - throw Exception( - DB::ErrorCodes::BAD_ARGUMENTS, - "Cannot insert values. {} key would be created setting the total keys number to {} exceeding the limit of {}", - created_nodes, - keys_num_after_insert, - keys_limit); - } - } - zookeeper->multi(requests); } @@ -290,31 +260,20 @@ public: } }; -namespace -{ - -zkutil::ZooKeeperPtr getZooKeeperClient(const std::string & hosts, const ContextPtr & context) -{ - if (hosts == default_host) - return context->getZooKeeper()->startNewSession(); - - return std::make_shared(hosts); -} - -} - StorageKeeperMap::StorageKeeperMap( - ContextPtr context, + ContextPtr context_, const StorageID & table_id, const StorageInMemoryMetadata & metadata, bool attach, std::string_view primary_key_, - std::string_view root_path_, - const std::string & hosts, - bool create_missing_root_path, - size_t keys_limit_, - bool remove_existing_data) - : IStorage(table_id), root_path(root_path_), primary_key(primary_key_), zookeeper_client(getZooKeeperClient(hosts, context)), log(&Poco::Logger::get("StorageKeeperMap")) + const std::string & root_path_, + bool create_missing_root_path) + : IStorage(table_id) + , WithContext(context_->getGlobalContext()) + , root_path(zkutil::extractZooKeeperPath(root_path_, false)) + , primary_key(primary_key_) + , zookeeper_name(zkutil::extractZooKeeperName(root_path_)) + , log(&Poco::Logger::get("StorageKeeperMap")) { setInMemoryMetadata(metadata); @@ -323,28 +282,21 @@ StorageKeeperMap::StorageKeeperMap( if (!root_path.starts_with('/')) throw Exception("root_path should start with '/'", ErrorCodes::BAD_ARGUMENTS); - auto client = getClient(); - std::filesystem::path root_path_fs{root_path}; - auto metadata_path_fs = root_path_fs / "__ch_metadata"; + auto metadata_path_fs = root_path_fs / "ch_metadata"; metadata_path = metadata_path_fs; lock_path = metadata_path_fs / "lock"; - auto keys_limit_path = metadata_path_fs / "keys_limit"; + tables_path = metadata_path_fs / "tables"; + dropped_path = metadata_path_fs / "dropped"; if (attach) { - // validate all metadata nodes are present - Coordination::Requests requests; - - requests.push_back(zkutil::makeCheckRequest(root_path, -1)); - requests.push_back(zkutil::makeCheckRequest(metadata_path, -1)); - requests.push_back(zkutil::makeCheckRequest(lock_path, -1)); - requests.push_back(zkutil::makeCheckRequest(keys_limit_path, -1)); - - client->multi(requests); + checkTable(); return; } + auto client = getClient(); + if (root_path != "/" && !client->exists(root_path)) { if (!create_missing_root_path) @@ -372,27 +324,62 @@ StorageKeeperMap::StorageKeeperMap( } } - Coordination::Stat stats; - auto exists = client->exists(root_path, &stats); - if (!exists) - throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Path '{}' should exist, but was deleted from another source", root_path); - if (stats.numChildren != 0) + ZooKeeperLock lock(lock_path, client, true); + + bool locked = false; + try { - if (!remove_existing_data) - throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Cannot create StorageKeeperMap using '{}' as path because it contains children nodes. Set remove_existing_data to 1 to clear children nodes", root_path); - - LOG_INFO(log, "Removing children for {} because remove_existing_data was set to true.", root_path); - client->removeChildrenRecursive(root_path); + lock.lock(); + locked = true; + } + catch (const Coordination::Exception & e) + { + if (e.code != Coordination::Error::ZNONODE) + throw; } - // create metadata nodes - Coordination::Requests create_requests; - create_requests.push_back(zkutil::makeCreateRequest(metadata_path, "", zkutil::CreateMode::Persistent)); - create_requests.push_back(zkutil::makeCreateRequest(lock_path, "", zkutil::CreateMode::Persistent)); - create_requests.push_back(zkutil::makeCreateRequest(keys_limit_path, toString(keys_limit_), zkutil::CreateMode::Persistent)); + { + Coordination::Stat stats; + auto exists = client->exists(root_path, &stats); + if (!exists) + throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Path '{}' should exist, but was deleted from another source", root_path); + } - client->multi(create_requests); + + const auto set_tables_num = [&] + { + std::string tables_num_str; + auto exists = client->tryGet(tables_path, tables_num_str); + + if (!exists) + { + client->create(tables_path, toString(1), zkutil::CreateMode::Persistent); + } + else + { + uint64_t tables_num = parse(tables_num_str); + client->set(tables_path, toString(tables_num + 1), zkutil::CreateMode::Persistent); + } + }; + + // we have metadata and lock nodes, because they can't be deleted with the lock acquired + if (locked) + { + if (client->exists(dropped_path)) + { + client->removeChildrenRecursive(root_path, getBaseName(metadata_path)); + client->remove(dropped_path); + } + + set_tables_num(); + return; + } + + client->createIfNotExists(metadata_path, ""); + client->createIfNotExists(lock_path, ""); + lock.lock(); + set_tables_num(); } @@ -400,11 +387,12 @@ Pipe StorageKeeperMap::read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, - ContextPtr context, + ContextPtr context_, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, unsigned num_streams) { + checkTable(); storage_snapshot->check(column_names); FieldVectorPtr filtered_keys; @@ -412,7 +400,7 @@ Pipe StorageKeeperMap::read( Block sample_block = storage_snapshot->metadata->getSampleBlock(); auto primary_key_type = sample_block.getByName(primary_key).type; - std::tie(filtered_keys, all_scan) = getFilterKeys(primary_key, primary_key_type, query_info, context); + std::tie(filtered_keys, all_scan) = getFilterKeys(primary_key, primary_key_type, query_info, context_); const auto process_keys = [&](KeyContainerPtr keys) -> Pipe { @@ -451,29 +439,62 @@ Pipe StorageKeeperMap::read( SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) { + checkTable(); return std::make_shared(*this, metadata_snapshot); } -void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr & , ContextPtr, TableExclusiveLockHolder &) +void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) { + checkTable(); auto client = getClient(); - ZooKeeperLock keeper_lock(lockPath(), client); - client->tryRemoveChildrenRecursive(root_path, /*probably_flat*/ true, getBaseName(metadata_path)); + ZooKeeperLock keeper_lock(lock_path, client); + client->removeChildrenRecursive(root_path, getBaseName(metadata_path)); } void StorageKeeperMap::drop() { + checkTable(); auto client = getClient(); - client->tryRemoveChildrenRecursive(root_path, /*probably_flat*/ false); + ZooKeeperLock lock(lock_path, client); + + auto tables_num_str = client->get(tables_path); + auto tables_num = parse(tables_num_str); + --tables_num; + + if (tables_num > 0) + { + client->set(tables_path, toString(tables_num)); + return; + } + + client->create(dropped_path, "", zkutil::CreateMode::Persistent); + + client->removeChildrenRecursive(root_path, getBaseName(metadata_path)); + + Coordination::Requests requests; + // we need to release lock and delete everything at the same time + // so create doesn't take a lock while delete is being run + auto current_lock_path = lock.release(); + requests.push_back(zkutil::makeRemoveRequest(current_lock_path, -1)); + requests.push_back(zkutil::makeRemoveRequest(lock_path, -1)); + requests.push_back(zkutil::makeRemoveRequest(tables_path, -1)); + requests.push_back(zkutil::makeRemoveRequest(dropped_path, -1)); + requests.push_back(zkutil::makeRemoveRequest(metadata_path, -1)); + + client->multi(requests); } zkutil::ZooKeeperPtr & StorageKeeperMap::getClient() const { - if (zookeeper_client->expired()) + std::lock_guard lock{zookeeper_mutex}; + if (!zookeeper_client || zookeeper_client->expired()) { - zookeeper_client = zookeeper_client->startNewSession(); - zookeeper_client->sync("/"); + zookeeper_client = nullptr; + if (zookeeper_name == "default") + zookeeper_client = getContext()->getZooKeeper(); + else + zookeeper_client = getContext()->getAuxiliaryZooKeeper(zookeeper_name); } return zookeeper_client; @@ -494,9 +515,45 @@ const std::string & StorageKeeperMap::lockPath() const return lock_path; } -UInt64 StorageKeeperMap::keysLimit() const +std::optional StorageKeeperMap::isTableValid() const { - return keys_limit; + std::lock_guard lock{init_mutex}; + if (table_is_valid.has_value()) + return *table_is_valid; + + try + { + // validate all metadata nodes are present + Coordination::Requests requests; + + requests.push_back(zkutil::makeCheckRequest(root_path, -1)); + requests.push_back(zkutil::makeCheckRequest(metadata_path, -1)); + requests.push_back(zkutil::makeCheckRequest(lock_path, -1)); + requests.push_back(zkutil::makeCheckRequest(tables_path, -1)); + + Coordination::Responses responses; + auto client = getClient(); + auto res = client->tryMulti(requests, responses); + table_is_valid = res == Coordination::Error::ZOK; + } + catch (const Coordination::Exception & e) + { + tryLogCurrentException(log); + + std::array retriable_errors{ + Coordination::Error::ZCONNECTIONLOSS, Coordination::Error::ZSESSIONEXPIRED, Coordination::Error::ZOPERATIONTIMEOUT}; + bool retriable_error + = std::any_of(retriable_errors.begin(), retriable_errors.end(), [&](const auto error) { return error == e.code; }); + if (!retriable_error) + table_is_valid = false; + } + catch (const Exception &) + { + tryLogCurrentException(log); + table_is_valid = false; + } + + return table_is_valid; } Chunk StorageKeeperMap::getByKeys(const ColumnsWithTypeAndName & keys, PaddedPODArray & null_map, const Names &) const @@ -586,53 +643,40 @@ Block StorageKeeperMap::getSampleBlock(const Names &) const namespace { -StoragePtr create(const StorageFactory::Arguments & args) -{ - ASTs & engine_args = args.engine_args; - if (engine_args.empty() || engine_args.size() > 4) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage KeeperMap requires 1-5 arguments:\n" - "root_path: path in the Keeper where the values will be stored (required)\n" - "hosts: comma separated Keeper hosts, set to '{0}' to use the same Keeper as ClickHouse (default: '{0}')\n" - "keys_limit: number of keys allowed, set to 0 for no limit (default: 0)\n" - "create_missing_root_path: 1 if the root path should be created if it's missing, otherwise throw exception (default: 1)\n", - "remove_existing_data: true if children inside 'root_path' should be deleted, otherwise throw exception (default: 0)", - default_host); + StoragePtr create(const StorageFactory::Arguments & args) + { + ASTs & engine_args = args.engine_args; + if (engine_args.empty() || engine_args.size() > 4) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage KeeperMap requires 1-5 arguments:\n" + "root_path: path in the Keeper where the values will be stored (required)\n" + "hosts: comma separated Keeper hosts, set to '{0}' to use the same Keeper as ClickHouse (default: '{0}')\n" + "create_missing_root_path: 1 if the root path should be created if it's missing, otherwise throw exception (default: 1)\n", + "remove_existing_data: true if children inside 'root_path' should be deleted, otherwise throw exception (default: 0)", + default_host); - auto root_path = checkAndGetLiteralArgument(engine_args[0], "root_path"); + auto root_path = checkAndGetLiteralArgument(engine_args[0], "root_path"); - std::string hosts = "default"; - if (engine_args.size() > 1) - hosts = checkAndGetLiteralArgument(engine_args[1], "hosts"); + bool create_missing_root_path = true; + if (engine_args.size() > 1) + create_missing_root_path = checkAndGetLiteralArgument(engine_args[1], "create_missing_root_path"); - size_t keys_limit = 0; - if (engine_args.size() > 2) - keys_limit = checkAndGetLiteralArgument(engine_args[2], "keys_limit"); + StorageInMemoryMetadata metadata; + metadata.setColumns(args.columns); + metadata.setConstraints(args.constraints); - bool create_missing_root_path = true; - if (engine_args.size() > 3) - create_missing_root_path = checkAndGetLiteralArgument(engine_args[3], "create_missing_root_path"); + if (!args.storage_def->primary_key) + throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); - bool remove_existing_data = false; - if (engine_args.size() > 4) - remove_existing_data = checkAndGetLiteralArgument(engine_args[4], "remove_existing_data"); + metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.getContext()); + auto primary_key_names = metadata.getColumnsRequiredForPrimaryKey(); + if (primary_key_names.size() != 1) + throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); - StorageInMemoryMetadata metadata; - metadata.setColumns(args.columns); - metadata.setConstraints(args.constraints); - - if (!args.storage_def->primary_key) - throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); - - metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.getContext()); - auto primary_key_names = metadata.getColumnsRequiredForPrimaryKey(); - if (primary_key_names.size() != 1) - throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); - - return std::make_shared( - args.getContext(), args.table_id, metadata, args.query.attach, primary_key_names[0], root_path, hosts, create_missing_root_path, keys_limit, remove_existing_data); -} + return std::make_shared( + args.getContext(), args.table_id, metadata, args.query.attach, primary_key_names[0], root_path, create_missing_root_path); + } } void registerStorageKeeperMap(StorageFactory & factory) diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 5f62d58bfa3..e09e8923e5c 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -14,21 +14,23 @@ namespace DB { +namespace ErrorCodes +{ + extern const int INVALID_STATE; +} + // KV store using (Zoo|CH)Keeper -class StorageKeeperMap final : public IStorage, public IKeyValueEntity +class StorageKeeperMap final : public IStorage, public IKeyValueEntity, WithContext { public: StorageKeeperMap( - ContextPtr context, + ContextPtr context_, const StorageID & table_id, const StorageInMemoryMetadata & metadata, bool attach, std::string_view primary_key_, - std::string_view root_path_, - const std::string & hosts, - bool create_missing_root_path, - size_t keys_limit, - bool remove_existing_data); + const std::string & root_path_, + bool create_missing_root_path); Pipe read( const Names & column_names, @@ -41,7 +43,7 @@ public: SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; - void truncate(const ASTPtr &, const StorageMetadataPtr & , ContextPtr, TableExclusiveLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; void drop() override; std::string getName() const override { return "KeeperMap"; } @@ -65,16 +67,54 @@ public: std::string fullPathForKey(std::string_view key) const; const std::string & lockPath() const; - UInt64 keysLimit() const; + std::optional isTableValid() const; + + template + void checkTable() const + { + auto is_table_valid = isTableValid(); + if (!is_table_valid.has_value()) + { + static constexpr std::string_view error_msg = "Failed to activate table because of connection issues. It will be activated " + "once a connection is established and metadata is verified"; + if constexpr (throw_on_error) + throw Exception(ErrorCodes::INVALID_STATE, error_msg); + else + { + LOG_ERROR(log, fmt::runtime(error_msg)); + return; + } + } + + if (!*is_table_valid) + { + static constexpr std::string_view error_msg + = "Failed to activate table because of invalid metadata in ZooKeeper. Please DETACH table"; + if constexpr (throw_on_error) + throw Exception(ErrorCodes::INVALID_STATE, error_msg); + else + { + LOG_ERROR(log, fmt::runtime(error_msg)); + return; + } + } + } private: std::string root_path; std::string primary_key; std::string metadata_path; std::string lock_path; - UInt64 keys_limit{0}; + std::string tables_path; + std::string dropped_path; - mutable zkutil::ZooKeeperPtr zookeeper_client; + std::string zookeeper_name; + + mutable std::mutex zookeeper_mutex; + mutable zkutil::ZooKeeperPtr zookeeper_client{nullptr}; + + mutable std::mutex init_mutex; + mutable std::optional table_is_valid; Poco::Logger * log; }; diff --git a/tests/queries/0_stateless/02381_keeper_map.sql b/tests/queries/0_stateless/02381_keeper_map.sql index 9a4f0b367f7..903d530299c 100644 --- a/tests/queries/0_stateless/02381_keeper_map.sql +++ b/tests/queries/0_stateless/02381_keeper_map.sql @@ -30,7 +30,6 @@ INSERT INTO 02381_test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), INSERT INTO 02381_test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; - SELECT A.a = B.a, A.b = B.b, A.c = B.c, A.d = B.d, A.e = B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 02381_test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 02381_test_memory) B USING a ORDER BY a; TRUNCATE TABLE 02381_test; From 41dd49356bb1965b97e5c03912c83971be1ee8ab Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 Aug 2022 17:27:07 +0000 Subject: [PATCH 11/43] Table create better --- src/Storages/StorageKeeperMap.cpp | 131 ++++++++++++++++-------------- src/Storages/StorageKeeperMap.h | 7 ++ 2 files changed, 77 insertions(+), 61 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 3477f29fc10..9be462d15a3 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -28,6 +28,7 @@ #include #include #include +#include "Core/UUID.h" #include #include @@ -275,6 +276,9 @@ StorageKeeperMap::StorageKeeperMap( , zookeeper_name(zkutil::extractZooKeeperName(root_path_)) , log(&Poco::Logger::get("StorageKeeperMap")) { + if (table_id.uuid == UUIDHelpers::Nil) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "KeeperMap cannot be used with '{}' database because UUID is needed. Please use Atomic or Replicated database", table_id.getDatabaseName()); + setInMemoryMetadata(metadata); if (root_path.empty()) @@ -287,7 +291,9 @@ StorageKeeperMap::StorageKeeperMap( metadata_path = metadata_path_fs; lock_path = metadata_path_fs / "lock"; tables_path = metadata_path_fs / "tables"; + table_path = fs::path(tables_path) / toString(table_id.uuid); dropped_path = metadata_path_fs / "dropped"; + dropped_lock_path = fs::path(dropped_path) / "lock"; if (attach) { @@ -309,77 +315,49 @@ StorageKeeperMap::StorageKeeperMap( else { LOG_TRACE(log, "Creating root path {}", root_path); - - size_t cur_pos = 0; - do - { - size_t search_start = cur_pos + 1; - cur_pos = root_path.find('/', search_start); - if (search_start == cur_pos) - throw Exception("root_path is invalid, contains subsequent '/'", ErrorCodes::BAD_ARGUMENTS); - - auto path = root_path.substr(0, cur_pos); - client->createIfNotExists(path, ""); - } while (cur_pos != std::string_view::npos); + client->createAncestors(root_path); + client->createIfNotExists(root_path, ""); } } - - ZooKeeperLock lock(lock_path, client, true); - - bool locked = false; - try - { - lock.lock(); - locked = true; - } - catch (const Coordination::Exception & e) - { - if (e.code != Coordination::Error::ZNONODE) - throw; - } - - { - Coordination::Stat stats; - auto exists = client->exists(root_path, &stats); - if (!exists) - throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Path '{}' should exist, but was deleted from another source", root_path); - } - - - const auto set_tables_num = [&] - { - std::string tables_num_str; - auto exists = client->tryGet(tables_path, tables_num_str); - - if (!exists) - { - client->create(tables_path, toString(1), zkutil::CreateMode::Persistent); - } - else - { - uint64_t tables_num = parse(tables_num_str); - client->set(tables_path, toString(tables_num + 1), zkutil::CreateMode::Persistent); - } - }; - - // we have metadata and lock nodes, because they can't be deleted with the lock acquired - if (locked) + for (size_t i = 0; i < 1000; ++i) { if (client->exists(dropped_path)) { - client->removeChildrenRecursive(root_path, getBaseName(metadata_path)); - client->remove(dropped_path); + LOG_INFO(log, "Removing leftover nodes"); + auto code = client->tryCreate(dropped_lock_path, "", zkutil::CreateMode::Ephemeral); + + if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) + { + LOG_INFO(log, "Someone else removed leftovers"); + } + else if (code != Coordination::Error::ZOK) + { + throw Coordination::Exception(code, dropped_lock_path); + } + else + { + auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(dropped_lock_path, *client); + if (!removeMetadataNodes(client, metadata_drop_lock)) + continue; + } } - set_tables_num(); - return; + client->createIfNotExists(metadata_path, ""); + client->createIfNotExists(tables_path, ""); + + auto code = client->tryCreate(table_path, "", zkutil::CreateMode::Persistent); + + if (code == Coordination::Error::ZOK) + return; + + if (code == Coordination::Error::ZNONODE) + LOG_INFO(log, "Metadata nodes were deleted in background, will retry"); + else + throw Coordination::Exception(code, table_path); } - client->createIfNotExists(metadata_path, ""); - client->createIfNotExists(lock_path, ""); - lock.lock(); - set_tables_num(); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot create metadata for table, becuase it is removed concurrently or becauyse of wrong root_path ({})", root_path); } @@ -452,6 +430,37 @@ void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont client->removeChildrenRecursive(root_path, getBaseName(metadata_path)); } +bool StorageKeeperMap::removeMetadataNodes(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock) +{ + bool completely_removed = false; + Coordination::Requests ops; + ops.emplace_back(zkutil::makeRemoveRequest(metadata_drop_lock->getPath(), -1)); + ops.emplace_back(zkutil::makeRemoveRequest(dropped_path, -1)); + ops.emplace_back(zkutil::makeRemoveRequest(metadata_path, -1)); + + Coordination::Responses responses; + auto code = zookeeper->tryMulti(ops, responses); + using enum Coordination::Error; + switch (code) + { + case ZOK: + { + metadata_drop_lock->setAlreadyRemoved(); + completely_removed = true; + LOG_INFO(log, "Metadata in {} was successfully removed from ZooKeeper", metadata_path); + break; + } + case ZNONODE: + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is a race condition between creation and removal of metadata. It's a bug"); + case ZNOTEMPTY: + LOG_ERROR(log, "Metadata was not completely removed from ZooKeeper"); + break; + default: + zkutil::KeeperMultiException::check(code, ops, responses); + } + return completely_removed; +} + void StorageKeeperMap::drop() { checkTable(); diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index e09e8923e5c..083d4c4c9ed 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -101,12 +101,19 @@ public: } private: + bool removeMetadataNodes(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock); + std::string root_path; std::string primary_key; + std::string metadata_path; std::string lock_path; + std::string tables_path; + std::string table_path; + std::string dropped_path; + std::string dropped_lock_path; std::string zookeeper_name; From 7c668be23a909d5c481d2c2806adca1b13472b42 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 25 Aug 2022 14:10:43 +0000 Subject: [PATCH 12/43] Fix drop --- src/Storages/StorageKeeperMap.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 9be462d15a3..f3917fe6d7d 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -357,7 +357,7 @@ StorageKeeperMap::StorageKeeperMap( throw Coordination::Exception(code, table_path); } - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot create metadata for table, becuase it is removed concurrently or becauyse of wrong root_path ({})", root_path); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot create metadata for table, becuase it is removed concurrently or because of wrong root_path ({})", root_path); } From 80365e0448e0e9d40f92b910cd7cd97ac864a957 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Aug 2022 09:19:59 +0000 Subject: [PATCH 13/43] Simplify drop and create --- src/Common/ZooKeeper/ZooKeeper.cpp | 10 +-- src/Storages/StorageKeeperMap.cpp | 127 +++++++---------------------- src/Storages/StorageKeeperMap.h | 3 +- 3 files changed, 35 insertions(+), 105 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index b47ab35820e..b95fa22ddb4 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -756,13 +756,13 @@ bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probab { String child_path = fs::path(path) / children.back(); - /// Will try to avoid recursive getChildren calls if child_path probably has no children. - /// It may be extremely slow when path contain a lot of leaf children. - if (!probably_flat) - tryRemoveChildrenRecursive(child_path); - if (likely(keep_child_node.empty() || keep_child_node != children.back())) { + /// Will try to avoid recursive getChildren calls if child_path probably has no children. + /// It may be extremely slow when path contain a lot of leaf children. + if (!probably_flat) + tryRemoveChildrenRecursive(child_path); + batch.push_back(child_path); ops.emplace_back(zkutil::makeRemoveRequest(child_path, -1)); } diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index f3917fe6d7d..362f4bfc152 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -24,6 +24,7 @@ #include #include "Common/Exception.h" +#include "Common/ZooKeeper/IKeeper.h" #include #include #include @@ -81,66 +82,6 @@ std::string_view getBaseName(const std::string_view path) return path.substr(last_slash + 1); } -struct ZooKeeperLock -{ - explicit ZooKeeperLock(std::string lock_path_, zkutil::ZooKeeperPtr client_, bool defer_lock = false) - : lock_path(std::move(lock_path_)), client(std::move(client_)) - { - if (!defer_lock) - lock(); - } - - ~ZooKeeperLock() - { - if (locked) - unlock(); - } - - void lock() - { - assert(!locked); - sequence_path = client->create(std::filesystem::path(lock_path) / "lock-", "", zkutil::CreateMode::EphemeralSequential); - auto node_name = getBaseName(sequence_path); - - while (true) - { - auto children = client->getChildren(lock_path); - assert(!children.empty()); - ::sort(children.begin(), children.end()); - - auto node_it = std::find(children.begin(), children.end(), node_name); - if (node_it == children.begin()) - { - locked = true; - return; - } - - client->waitForDisappear(*(node_it - 1)); - } - } - - void unlock() - { - assert(locked); - client->tryRemove(sequence_path); - } - - // release the ownership and return the path - // for the lock - std::string release() - { - assert(locked); - locked = false; - return std::move(sequence_path); - } - -private: - std::string lock_path; - std::string sequence_path; - zkutil::ZooKeeperPtr client; - bool locked{false}; -}; - } class StorageKeeperMapSink : public SinkToStorage @@ -289,7 +230,6 @@ StorageKeeperMap::StorageKeeperMap( std::filesystem::path root_path_fs{root_path}; auto metadata_path_fs = root_path_fs / "ch_metadata"; metadata_path = metadata_path_fs; - lock_path = metadata_path_fs / "lock"; tables_path = metadata_path_fs / "tables"; table_path = fs::path(tables_path) / toString(table_id.uuid); dropped_path = metadata_path_fs / "dropped"; @@ -425,9 +365,7 @@ void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont { checkTable(); auto client = getClient(); - - ZooKeeperLock keeper_lock(lock_path, client); - client->removeChildrenRecursive(root_path, getBaseName(metadata_path)); + client->tryRemoveChildrenRecursive(root_path, true, getBaseName(metadata_path)); } bool StorageKeeperMap::removeMetadataNodes(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock) @@ -457,6 +395,7 @@ bool StorageKeeperMap::removeMetadataNodes(zkutil::ZooKeeperPtr zookeeper, const break; default: zkutil::KeeperMultiException::check(code, ops, responses); + break; } return completely_removed; } @@ -465,33 +404,38 @@ void StorageKeeperMap::drop() { checkTable(); auto client = getClient(); - ZooKeeperLock lock(lock_path, client); - auto tables_num_str = client->get(tables_path); - auto tables_num = parse(tables_num_str); - --tables_num; + client->remove(table_path); - if (tables_num > 0) + if (!client->getChildren(tables_path).empty()) + return; + + Coordination::Requests ops; + Coordination::Responses responses; + + ops.emplace_back(zkutil::makeRemoveRequest(tables_path, -1)); + ops.emplace_back(zkutil::makeCreateRequest(dropped_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(dropped_lock_path, "", zkutil::CreateMode::Ephemeral)); + + auto code = client->tryMulti(ops, responses); + + if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) { - client->set(tables_path, toString(tables_num)); + LOG_INFO(log, "Metadata is being removed by another table"); return; } - - client->create(dropped_path, "", zkutil::CreateMode::Persistent); + else if (code == Coordination::Error::ZNOTEMPTY) + { + LOG_WARNING(log, "Another table is using the same path, metadata will not be deleted"); + return; + } + else if (code != Coordination::Error::ZOK) + zkutil::KeeperMultiException::check(code, ops, responses); client->removeChildrenRecursive(root_path, getBaseName(metadata_path)); - Coordination::Requests requests; - // we need to release lock and delete everything at the same time - // so create doesn't take a lock while delete is being run - auto current_lock_path = lock.release(); - requests.push_back(zkutil::makeRemoveRequest(current_lock_path, -1)); - requests.push_back(zkutil::makeRemoveRequest(lock_path, -1)); - requests.push_back(zkutil::makeRemoveRequest(tables_path, -1)); - requests.push_back(zkutil::makeRemoveRequest(dropped_path, -1)); - requests.push_back(zkutil::makeRemoveRequest(metadata_path, -1)); - - client->multi(requests); + auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(dropped_lock_path, *client); + removeMetadataNodes(client, metadata_drop_lock); } zkutil::ZooKeeperPtr & StorageKeeperMap::getClient() const @@ -519,11 +463,6 @@ std::string StorageKeeperMap::fullPathForKey(const std::string_view key) const return fmt::format("{}/{}", root_path, key); } -const std::string & StorageKeeperMap::lockPath() const -{ - return lock_path; -} - std::optional StorageKeeperMap::isTableValid() const { std::lock_guard lock{init_mutex}; @@ -534,11 +473,7 @@ std::optional StorageKeeperMap::isTableValid() const { // validate all metadata nodes are present Coordination::Requests requests; - - requests.push_back(zkutil::makeCheckRequest(root_path, -1)); - requests.push_back(zkutil::makeCheckRequest(metadata_path, -1)); - requests.push_back(zkutil::makeCheckRequest(lock_path, -1)); - requests.push_back(zkutil::makeCheckRequest(tables_path, -1)); + requests.push_back(zkutil::makeCheckRequest(table_path, -1)); Coordination::Responses responses; auto client = getClient(); @@ -549,11 +484,7 @@ std::optional StorageKeeperMap::isTableValid() const { tryLogCurrentException(log); - std::array retriable_errors{ - Coordination::Error::ZCONNECTIONLOSS, Coordination::Error::ZSESSIONEXPIRED, Coordination::Error::ZOPERATIONTIMEOUT}; - bool retriable_error - = std::any_of(retriable_errors.begin(), retriable_errors.end(), [&](const auto error) { return error == e.code; }); - if (!retriable_error) + if (!Coordination::isHardwareError(e.code)) table_is_valid = false; } catch (const Exception &) diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 083d4c4c9ed..22c5bb66e1a 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -103,11 +103,10 @@ public: private: bool removeMetadataNodes(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock); - std::string root_path; + const std::string root_path; std::string primary_key; std::string metadata_path; - std::string lock_path; std::string tables_path; std::string table_path; From 30429d75b383c38a61f75e8463c377428f2fbfd2 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Aug 2022 10:22:35 +0000 Subject: [PATCH 14/43] Add basic tests for KeeperMap --- .../queries/0_stateless/02381_keeper_map.sql | 39 ------------------- ...p.reference => 02416_keeper_map.reference} | 0 .../queries/0_stateless/02416_keeper_map.sql | 37 ++++++++++++++++++ .../02417_keeper_map_create_drop.reference | 11 ++++++ .../02417_keeper_map_create_drop.sql | 20 ++++++++++ 5 files changed, 68 insertions(+), 39 deletions(-) delete mode 100644 tests/queries/0_stateless/02381_keeper_map.sql rename tests/queries/0_stateless/{02381_keeper_map.reference => 02416_keeper_map.reference} (100%) create mode 100644 tests/queries/0_stateless/02416_keeper_map.sql create mode 100644 tests/queries/0_stateless/02417_keeper_map_create_drop.reference create mode 100644 tests/queries/0_stateless/02417_keeper_map_create_drop.sql diff --git a/tests/queries/0_stateless/02381_keeper_map.sql b/tests/queries/0_stateless/02381_keeper_map.sql deleted file mode 100644 index 903d530299c..00000000000 --- a/tests/queries/0_stateless/02381_keeper_map.sql +++ /dev/null @@ -1,39 +0,0 @@ --- Tags: no-fasttest - -SET database_atomic_wait_for_drop_and_detach_synchronously = 1; - -DROP TABLE IF EXISTS 02381_test; - -CREATE TABLE 02381_test (key String, value UInt32) Engine=KeeperMap('/test2381'); -- { serverError 36 } -CREATE TABLE 02381_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key2); -- { serverError 47 } -CREATE TABLE 02381_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key, value); -- { serverError 36 } -CREATE TABLE 02381_test (key Tuple(String, UInt32), value UInt64) Engine=KeeperMap('/test2381') PRIMARY KEY(key); - -DROP TABLE IF EXISTS 02381_test; -CREATE TABLE 02381_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key); - -INSERT INTO 02381_test SELECT '1_1', number FROM numbers(10000); -SELECT COUNT(1) == 1 FROM 02381_test; - -INSERT INTO 02381_test SELECT concat(toString(number), '_1'), number FROM numbers(10000); -SELECT COUNT(1) == 10000 FROM 02381_test; -SELECT uniqExact(key) == 32 FROM (SELECT * FROM 02381_test LIMIT 32 SETTINGS max_block_size = 1); -SELECT SUM(value) == 1 + 99 + 900 FROM 02381_test WHERE key IN ('1_1', '99_1', '900_1'); - -DROP TABLE IF EXISTS 02381_test; -DROP TABLE IF EXISTS 02381_test_memory; - -CREATE TABLE 02381_test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=KeeperMap('/test2381') PRIMARY KEY(k); -CREATE TABLE 02381_test_memory AS 02381_test Engine = Memory; - -INSERT INTO 02381_test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; - -INSERT INTO 02381_test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; - -SELECT A.a = B.a, A.b = B.b, A.c = B.c, A.d = B.d, A.e = B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 02381_test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 02381_test_memory) B USING a ORDER BY a; - -TRUNCATE TABLE 02381_test; -SELECT 0 == COUNT(1) FROM 02381_test; - -DROP TABLE IF EXISTS 02381_test; -DROP TABLE IF EXISTS 02381_test_memory; diff --git a/tests/queries/0_stateless/02381_keeper_map.reference b/tests/queries/0_stateless/02416_keeper_map.reference similarity index 100% rename from tests/queries/0_stateless/02381_keeper_map.reference rename to tests/queries/0_stateless/02416_keeper_map.reference diff --git a/tests/queries/0_stateless/02416_keeper_map.sql b/tests/queries/0_stateless/02416_keeper_map.sql new file mode 100644 index 00000000000..6688f2b4659 --- /dev/null +++ b/tests/queries/0_stateless/02416_keeper_map.sql @@ -0,0 +1,37 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS 02416_test SYNC; + +CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/test2381'); -- { serverError 36 } +CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key2); -- { serverError 47 } +CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key, value); -- { serverError 36 } +CREATE TABLE 02416_test (key Tuple(String, UInt32), value UInt64) Engine=KeeperMap('/test2381') PRIMARY KEY(key); + +DROP TABLE IF EXISTS 02416_test SYNC; +CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key); + +INSERT INTO 02416_test SELECT '1_1', number FROM numbers(10000); +SELECT COUNT(1) == 1 FROM 02416_test; + +INSERT INTO 02416_test SELECT concat(toString(number), '_1'), number FROM numbers(10000); +SELECT COUNT(1) == 10000 FROM 02416_test; +SELECT uniqExact(key) == 32 FROM (SELECT * FROM 02416_test LIMIT 32 SETTINGS max_block_size = 1); +SELECT SUM(value) == 1 + 99 + 900 FROM 02416_test WHERE key IN ('1_1', '99_1', '900_1'); + +DROP TABLE IF EXISTS 02416_test SYNC; +DROP TABLE IF EXISTS 02416_test_memory; + +CREATE TABLE 02416_test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=KeeperMap('/test2381') PRIMARY KEY(k); +CREATE TABLE 02416_test_memory AS 02416_test Engine = Memory; + +INSERT INTO 02416_test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; + +INSERT INTO 02416_test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; + +SELECT A.a = B.a, A.b = B.b, A.c = B.c, A.d = B.d, A.e = B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 02416_test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 02416_test_memory) B USING a ORDER BY a; + +TRUNCATE TABLE 02416_test; +SELECT 0 == COUNT(1) FROM 02416_test; + +DROP TABLE IF EXISTS 02416_test SYNC; +DROP TABLE IF EXISTS 02416_test_memory; diff --git a/tests/queries/0_stateless/02417_keeper_map_create_drop.reference b/tests/queries/0_stateless/02417_keeper_map_create_drop.reference new file mode 100644 index 00000000000..dda215c9659 --- /dev/null +++ b/tests/queries/0_stateless/02417_keeper_map_create_drop.reference @@ -0,0 +1,11 @@ +1 11 +2 22 +------ +1 11 +2 22 +------ +1 11 +2 22 +------ +1 11 +2 22 diff --git a/tests/queries/0_stateless/02417_keeper_map_create_drop.sql b/tests/queries/0_stateless/02417_keeper_map_create_drop.sql new file mode 100644 index 00000000000..fff1d78d589 --- /dev/null +++ b/tests/queries/0_stateless/02417_keeper_map_create_drop.sql @@ -0,0 +1,20 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS 02417_test SYNC; + +CREATE TABLE 02417_test (key UInt64, value UInt64) Engine=KeeperMap('/test2417') PRIMARY KEY(key); +INSERT INTO 02417_test VALUES (1, 11); +SELECT * FROM 02417_test ORDER BY key; +SELECT '------'; + +CREATE TABLE 02417_test_another (key UInt64, value UInt64) Engine=KeeperMap('/test2417') PRIMARY KEY(key); +INSERT INTO 02417_test_another VALUES (2, 22); +SELECT * FROM 02417_test_another ORDER BY key; +SELECT '------'; +SELECT * FROM 02417_test ORDER BY key; +SELECT '------'; + +DROP TABLE 02417_test SYNC; +SELECT * FROM 02417_test_another ORDER BY key; + +DROP TABLE 02417_test_another SYNC; From bf03dfc661a336875b19c440a917ae120ad95d19 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Aug 2022 13:41:13 +0000 Subject: [PATCH 15/43] Fix create --- src/Storages/StorageKeeperMap.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 362f4bfc152..e6090ccc605 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -289,7 +289,10 @@ StorageKeeperMap::StorageKeeperMap( auto code = client->tryCreate(table_path, "", zkutil::CreateMode::Persistent); if (code == Coordination::Error::ZOK) + { + table_is_valid = true; return; + } if (code == Coordination::Error::ZNONODE) LOG_INFO(log, "Metadata nodes were deleted in background, will retry"); From 3c098bd0cb7303aa8968ad995303608a53719262 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Aug 2022 13:41:36 +0000 Subject: [PATCH 16/43] Add integration test for KeeperMap --- tests/integration/test_keeper_map/__init__.py | 1 + tests/integration/test_keeper_map/test.py | 172 ++++++++++++++++++ 2 files changed, 173 insertions(+) create mode 100644 tests/integration/test_keeper_map/__init__.py create mode 100644 tests/integration/test_keeper_map/test.py diff --git a/tests/integration/test_keeper_map/__init__.py b/tests/integration/test_keeper_map/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_keeper_map/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_keeper_map/test.py b/tests/integration/test_keeper_map/test.py new file mode 100644 index 00000000000..025439f44db --- /dev/null +++ b/tests/integration/test_keeper_map/test.py @@ -0,0 +1,172 @@ +import multiprocessing +import pytest +from time import sleep +import random +from itertools import count +from sys import stdout + +from multiprocessing import Pool + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry, assert_logs_contain +from helpers.network import PartitionManager + +test_recover_staled_replica_run = 1 + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + main_configs=[], + with_zookeeper=True, + stay_alive=True, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def get_genuine_zk(): + print("Zoo1", cluster.get_instance_ip("zoo1")) + return cluster.get_kazoo_client("zoo1") + + +def remove_children(client, path): + children = client.get_children(path) + + for child in children: + child_path = f"{path}/{child}" + remove_children(client, child_path) + client.delete(child_path) + + +def test_create_keeper_map(started_cluster): + assert "Path '/test1' doesn't exist" in node.query_and_get_error( + "CREATE TABLE test_keeper_map (key UInt64, value UInt64) ENGINE = KeeperMap('/test1', 0) PRIMARY KEY(key);" + ) + + node.query( + "CREATE TABLE test_keeper_map (key UInt64, value UInt64) ENGINE = KeeperMap('/test1') PRIMARY KEY(key);" + ) + zk_client = get_genuine_zk() + + def assert_children_size(expected_size): + assert len(zk_client.get_children("/test1")) == expected_size + + assert_children_size(1) + + node.query("INSERT INTO test_keeper_map VALUES (1, 11)") + assert_children_size(2) + + node.query( + "CREATE TABLE test_keeper_map_another (key UInt64, value UInt64) ENGINE = KeeperMap('/test1') PRIMARY KEY(key);" + ) + assert_children_size(2) + node.query("INSERT INTO test_keeper_map_another VALUES (1, 11)") + assert_children_size(2) + + node.query("INSERT INTO test_keeper_map_another VALUES (2, 22)") + assert_children_size(3) + + node.query("DROP TABLE test_keeper_map SYNC") + assert_children_size(3) + + node.query("DROP TABLE test_keeper_map_another SYNC") + assert_children_size(0) + + zk_client.stop() + + +def create_drop_loop(index, stop_event): + table_name = f"test_keeper_map_{index}" + + for i in count(0, 1): + if stop_event.is_set(): + return + + stdout.write(f"Trying with {i} for {index}\n") + node.query( + f"CREATE TABLE {table_name} (key UInt64, value UInt64) ENGINE = KeeperMap('/test') PRIMARY KEY(key);" + ) + node.query(f"INSERT INTO {table_name} VALUES ({index}, {i})") + result = node.query(f"SELECT value FROM {table_name} WHERE key = {index}") + assert result.strip() == str(i) + node.query(f"DROP TABLE {table_name} SYNC") + + +def test_create_drop_keeper_map_concurrent(started_cluster): + pool = Pool() + manager = multiprocessing.Manager() + stop_event = manager.Event() + results = [] + for i in range(multiprocessing.cpu_count()): + sleep(0.2) + results.append( + pool.apply_async( + create_drop_loop, + args=( + i, + stop_event, + ), + ) + ) + + sleep(60) + stop_event.set() + + for result in results: + result.get() + + pool.close() + + client = get_genuine_zk() + assert len(client.get_children("/test")) == 0 + + +def test_keeper_map_without_zk(started_cluster): + def assert_keeper_exception_after_partition(query): + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node) + error = node.query_and_get_error(query) + assert "Coordination::Exception" in error + + assert_keeper_exception_after_partition( + "CREATE TABLE test_keeper_map (key UInt64, value UInt64) ENGINE = KeeperMap('/test1') PRIMARY KEY(key);" + ) + + node.query( + "CREATE TABLE test_keeper_map (key UInt64, value UInt64) ENGINE = KeeperMap('/test1') PRIMARY KEY(key);" + ) + + assert_keeper_exception_after_partition( + "INSERT INTO test_keeper_map VALUES (1, 11)" + ) + node.query("INSERT INTO test_keeper_map VALUES (1, 11)") + + assert_keeper_exception_after_partition("SELECT * FROM test_keeper_map") + node.query("SELECT * FROM test_keeper_map") + + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node) + node.restart_clickhouse(60) + error = node.query_and_get_error("SELECT * FROM test_keeper_map") + assert "Failed to activate table because of connection issues" in error + + node.query("SELECT * FROM test_keeper_map") + + client = get_genuine_zk() + remove_children(client, "/test1") + node.restart_clickhouse(60) + error = node.query_and_get_error("SELECT * FROM test_keeper_map") + assert "Failed to activate table because of invalid metadata in ZooKeeper" in error + + node.query("DETACH TABLE test_keeper_map") + + client.stop() From d498213973ef1d1ca7a1ac35979e4ab17fa7c248 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 Aug 2022 13:50:02 +0000 Subject: [PATCH 17/43] Style fixes --- src/Storages/StorageKeeperMap.cpp | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index e6090ccc605..dd55586208e 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -296,11 +296,11 @@ StorageKeeperMap::StorageKeeperMap( if (code == Coordination::Error::ZNONODE) LOG_INFO(log, "Metadata nodes were deleted in background, will retry"); - else + else throw Coordination::Exception(code, table_path); } - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot create metadata for table, becuase it is removed concurrently or because of wrong root_path ({})", root_path); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot create metadata for table, because it is removed concurrently or because of wrong root_path ({})", root_path); } @@ -589,14 +589,12 @@ namespace StoragePtr create(const StorageFactory::Arguments & args) { ASTs & engine_args = args.engine_args; - if (engine_args.empty() || engine_args.size() > 4) + if (engine_args.empty() || engine_args.size() > 2) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage KeeperMap requires 1-5 arguments:\n" + "Storage KeeperMap requires 1-2 arguments:\n" "root_path: path in the Keeper where the values will be stored (required)\n" - "hosts: comma separated Keeper hosts, set to '{0}' to use the same Keeper as ClickHouse (default: '{0}')\n" "create_missing_root_path: 1 if the root path should be created if it's missing, otherwise throw exception (default: 1)\n", - "remove_existing_data: true if children inside 'root_path' should be deleted, otherwise throw exception (default: 0)", default_host); auto root_path = checkAndGetLiteralArgument(engine_args[0], "root_path"); From e97640e2f0f1d52c1e48db22c7629eecc980295f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 31 Aug 2022 07:21:20 +0000 Subject: [PATCH 18/43] Fix reference for keeper test --- tests/queries/0_stateless/02417_keeper_map_create_drop.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02417_keeper_map_create_drop.reference b/tests/queries/0_stateless/02417_keeper_map_create_drop.reference index dda215c9659..25bc8c288fb 100644 --- a/tests/queries/0_stateless/02417_keeper_map_create_drop.reference +++ b/tests/queries/0_stateless/02417_keeper_map_create_drop.reference @@ -1,5 +1,4 @@ 1 11 -2 22 ------ 1 11 2 22 From 5ef4c52290347f6936b392f74f2a026a45e750cf Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 31 Aug 2022 08:14:28 +0000 Subject: [PATCH 19/43] Add support for keys limit --- src/Storages/StorageKeeperMap.cpp | 124 ++++++++++++++++++++++-------- src/Storages/StorageKeeperMap.h | 12 ++- 2 files changed, 102 insertions(+), 34 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index dd55586208e..70f32bbcd74 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -30,6 +30,7 @@ #include #include #include "Core/UUID.h" +#include "base/types.h" #include #include @@ -46,6 +47,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int KEEPER_EXCEPTION; extern const int LOGICAL_ERROR; + extern const int LIMIT_EXCEEDED; } namespace @@ -128,18 +130,48 @@ public: void onFinish() override { - auto & zookeeper = storage.getClient(); + auto zookeeper = storage.getClient(); Coordination::Requests requests; + auto keys_limit = storage.keysLimit(); + + size_t current_keys_num = 0; + size_t new_keys_num = 0; + + if (keys_limit != 0) + { + Coordination::Stat root_stat; + zookeeper->get(storage.rootKeeperPath(), &root_stat); + // exclude metadata node + current_keys_num = root_stat.numChildren - 1; + } + for (const auto & [key, value] : new_values) { auto path = storage.fullPathForKey(key); if (zookeeper->exists(path)) + { requests.push_back(zkutil::makeSetRequest(path, value, -1)); + } else + { requests.push_back(zkutil::makeCreateRequest(path, value, zkutil::CreateMode::Persistent)); + ++new_keys_num; + } + } + + if (new_keys_num != 0) + { + auto will_be = current_keys_num + new_keys_num; + if (keys_limit != 0 && will_be > keys_limit) + throw Exception( + ErrorCodes::LIMIT_EXCEEDED, + "Limit would be exceeded by inserting {} new key(s). Limit is {}, while the number of keys would be {}", + new_keys_num, + keys_limit, + will_be); } zookeeper->multi(requests); @@ -209,12 +241,14 @@ StorageKeeperMap::StorageKeeperMap( bool attach, std::string_view primary_key_, const std::string & root_path_, - bool create_missing_root_path) + bool create_missing_root_path, + UInt64 keys_limit_) : IStorage(table_id) , WithContext(context_->getGlobalContext()) , root_path(zkutil::extractZooKeeperPath(root_path_, false)) , primary_key(primary_key_) , zookeeper_name(zkutil::extractZooKeeperName(root_path_)) + , keys_limit(keys_limit_) , log(&Poco::Logger::get("StorageKeeperMap")) { if (table_id.uuid == UUIDHelpers::Nil) @@ -227,6 +261,24 @@ StorageKeeperMap::StorageKeeperMap( if (!root_path.starts_with('/')) throw Exception("root_path should start with '/'", ErrorCodes::BAD_ARGUMENTS); + + auto config_keys_limit = context_->getConfigRef().getUInt64("keeper_map_keys_limit", 0); + if (config_keys_limit != 0 && keys_limit > config_keys_limit) + { + LOG_WARNING( + log, + "Keys limit for {} defined by argument ({}) is larger than the one defined by 'keeper_map_keys_limit' config ({}). Will use " + "config defined value", + getStorageID().getFullTableName(), + keys_limit, + config_keys_limit); + keys_limit = config_keys_limit; + } + else if (keys_limit > 0) + { + LOG_INFO(log, "Keys limit for {} will be set to {}", getStorageID().getFullTableName(), keys_limit); + } + std::filesystem::path root_path_fs{root_path}; auto metadata_path_fs = root_path_fs / "ch_metadata"; metadata_path = metadata_path_fs; @@ -351,7 +403,7 @@ Pipe StorageKeeperMap::read( return Pipe::unitePipes(std::move(pipes)); }; - auto & client = getClient(); + auto client = getClient(); if (all_scan) return process_keys(std::make_shared>(client->getChildren(root_path))); @@ -441,7 +493,7 @@ void StorageKeeperMap::drop() removeMetadataNodes(client, metadata_drop_lock); } -zkutil::ZooKeeperPtr & StorageKeeperMap::getClient() const +zkutil::ZooKeeperPtr StorageKeeperMap::getClient() const { std::lock_guard lock{zookeeper_mutex}; if (!zookeeper_client || zookeeper_client->expired()) @@ -466,6 +518,11 @@ std::string StorageKeeperMap::fullPathForKey(const std::string_view key) const return fmt::format("{}/{}", root_path, key); } +UInt64 StorageKeeperMap::keysLimit() const +{ + return keys_limit; +} + std::optional StorageKeeperMap::isTableValid() const { std::lock_guard lock{init_mutex}; @@ -586,38 +643,45 @@ Block StorageKeeperMap::getSampleBlock(const Names &) const namespace { - StoragePtr create(const StorageFactory::Arguments & args) - { - ASTs & engine_args = args.engine_args; - if (engine_args.empty() || engine_args.size() > 2) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage KeeperMap requires 1-2 arguments:\n" - "root_path: path in the Keeper where the values will be stored (required)\n" - "create_missing_root_path: 1 if the root path should be created if it's missing, otherwise throw exception (default: 1)\n", - default_host); - auto root_path = checkAndGetLiteralArgument(engine_args[0], "root_path"); +StoragePtr create(const StorageFactory::Arguments & args) +{ + ASTs & engine_args = args.engine_args; + if (engine_args.empty() || engine_args.size() > 3) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage KeeperMap requires 1-2 arguments:\n" + "root_path: path in the Keeper where the values will be stored (required)\n" + "create_missing_root_path: 1 if the root path should be created if it's missing, otherwise throw exception (default: 1)\n", + "keys_limit: number of keys allowed to be stored, 0 is no limit (default: 0)\n", + default_host); - bool create_missing_root_path = true; - if (engine_args.size() > 1) - create_missing_root_path = checkAndGetLiteralArgument(engine_args[1], "create_missing_root_path"); + auto root_path = checkAndGetLiteralArgument(engine_args[0], "root_path"); - StorageInMemoryMetadata metadata; - metadata.setColumns(args.columns); - metadata.setConstraints(args.constraints); + bool create_missing_root_path = true; + if (engine_args.size() > 1) + create_missing_root_path = checkAndGetLiteralArgument(engine_args[1], "create_missing_root_path"); - if (!args.storage_def->primary_key) - throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); + UInt64 keys_limit = 0; + if (engine_args.size() > 2) + keys_limit = checkAndGetLiteralArgument(engine_args[2], "keys_limit"); - metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.getContext()); - auto primary_key_names = metadata.getColumnsRequiredForPrimaryKey(); - if (primary_key_names.size() != 1) - throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); + StorageInMemoryMetadata metadata; + metadata.setColumns(args.columns); + metadata.setConstraints(args.constraints); + + if (!args.storage_def->primary_key) + throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); + + metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.getContext()); + auto primary_key_names = metadata.getColumnsRequiredForPrimaryKey(); + if (primary_key_names.size() != 1) + throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); + + return std::make_shared( + args.getContext(), args.table_id, metadata, args.query.attach, primary_key_names[0], root_path, create_missing_root_path, keys_limit); +} - return std::make_shared( - args.getContext(), args.table_id, metadata, args.query.attach, primary_key_names[0], root_path, create_missing_root_path); - } } void registerStorageKeeperMap(StorageFactory & factory) diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 22c5bb66e1a..fa3a9e4bd2e 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -30,7 +30,8 @@ public: bool attach, std::string_view primary_key_, const std::string & root_path_, - bool create_missing_root_path); + bool create_missing_root_path, + UInt64 keys_limit_); Pipe read( const Names & column_names, @@ -62,12 +63,11 @@ public: return node->getColumnName() == primary_key; } - zkutil::ZooKeeperPtr & getClient() const; + zkutil::ZooKeeperPtr getClient() const; const std::string & rootKeeperPath() const; std::string fullPathForKey(std::string_view key) const; - const std::string & lockPath() const; - std::optional isTableValid() const; + UInt64 keysLimit() const; template void checkTable() const @@ -103,6 +103,8 @@ public: private: bool removeMetadataNodes(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock); + std::optional isTableValid() const; + const std::string root_path; std::string primary_key; @@ -116,6 +118,8 @@ private: std::string zookeeper_name; + uint64_t keys_limit{0}; + mutable std::mutex zookeeper_mutex; mutable zkutil::ZooKeeperPtr zookeeper_client{nullptr}; From c73f41868bd0059b0f235e8f933d049821c79e66 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 31 Aug 2022 08:14:50 +0000 Subject: [PATCH 20/43] Add tests for keys_limit --- .../02418_keeper_map_keys_limit.reference | 4 ++++ .../02418_keeper_map_keys_limit.sql | 23 +++++++++++++++++++ 2 files changed, 27 insertions(+) create mode 100644 tests/queries/0_stateless/02418_keeper_map_keys_limit.reference create mode 100644 tests/queries/0_stateless/02418_keeper_map_keys_limit.sql diff --git a/tests/queries/0_stateless/02418_keeper_map_keys_limit.reference b/tests/queries/0_stateless/02418_keeper_map_keys_limit.reference new file mode 100644 index 00000000000..95c45d6da51 --- /dev/null +++ b/tests/queries/0_stateless/02418_keeper_map_keys_limit.reference @@ -0,0 +1,4 @@ +2 +3 +4 +4 diff --git a/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql b/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql new file mode 100644 index 00000000000..aedeba661dd --- /dev/null +++ b/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql @@ -0,0 +1,23 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS 02418_test SYNC; + +CREATE TABLE 02418_test (key UInt64, value Float64) Engine=KeeperMap('/test2418', 1, 3) PRIMARY KEY(key); + +INSERT INTO 02418_test VALUES (1, 1.1), (2, 2.2); +SELECT count() FROM 02418_test; + +INSERT INTO 02418_test VALUES (3, 3.3), (4, 4.4); -- { serverError 290 } + +INSERT INTO 02418_test VALUES (1, 2.1), (2, 3.2), (3, 3.3); +SELECT count() FROM 02418_test; + +CREATE TABLE 02418_test_another (key UInt64, value Float64) Engine=KeeperMap('/test2418', 1, 4) PRIMARY KEY(key); +INSERT INTO 02418_test VALUES (4, 4.4); -- { serverError 290 } +INSERT INTO 02418_test_another VALUES (4, 4.4); + +SELECT count() FROM 02418_test; +SELECT count() FROM 02418_test_another; + +DROP TABLE 02418_test SYNC; +DROP TABLE 02418_test_another SYNC; From 1ff14aed023a02ea9e0fd365c4edacab82f07e70 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 31 Aug 2022 08:23:44 +0000 Subject: [PATCH 21/43] Fix dropping table --- src/Storages/StorageKeeperMap.cpp | 10 +++++----- src/Storages/StorageKeeperMap.h | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 70f32bbcd74..66b7c1731f1 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -330,7 +330,7 @@ StorageKeeperMap::StorageKeeperMap( else { auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(dropped_lock_path, *client); - if (!removeMetadataNodes(client, metadata_drop_lock)) + if (!dropTable(client, metadata_drop_lock)) continue; } } @@ -423,8 +423,10 @@ void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont client->tryRemoveChildrenRecursive(root_path, true, getBaseName(metadata_path)); } -bool StorageKeeperMap::removeMetadataNodes(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock) +bool StorageKeeperMap::dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock) { + zookeeper->removeChildrenRecursive(root_path, getBaseName(metadata_path)); + bool completely_removed = false; Coordination::Requests ops; ops.emplace_back(zkutil::makeRemoveRequest(metadata_drop_lock->getPath(), -1)); @@ -487,10 +489,8 @@ void StorageKeeperMap::drop() else if (code != Coordination::Error::ZOK) zkutil::KeeperMultiException::check(code, ops, responses); - client->removeChildrenRecursive(root_path, getBaseName(metadata_path)); - auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(dropped_lock_path, *client); - removeMetadataNodes(client, metadata_drop_lock); + dropTable(client, metadata_drop_lock); } zkutil::ZooKeeperPtr StorageKeeperMap::getClient() const diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index fa3a9e4bd2e..3e47e5ee9c6 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -101,7 +101,7 @@ public: } private: - bool removeMetadataNodes(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock); + bool dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock); std::optional isTableValid() const; From 1af9a81297e224ddad324d908669a60f375df044 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 31 Aug 2022 08:29:40 +0000 Subject: [PATCH 22/43] Sync after reconnect --- src/Storages/StorageKeeperMap.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 66b7c1731f1..157b35ce995 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -503,6 +503,8 @@ zkutil::ZooKeeperPtr StorageKeeperMap::getClient() const zookeeper_client = getContext()->getZooKeeper(); else zookeeper_client = getContext()->getAuxiliaryZooKeeper(zookeeper_name); + + zookeeper_client->sync(rootKeeperPath()); } return zookeeper_client; From 7d5e13c6ba945ba3f2e5ab2675cc4dc7586db850 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 31 Aug 2022 10:40:32 +0000 Subject: [PATCH 23/43] Use async exist for write --- src/Storages/StorageKeeperMap.cpp | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 157b35ce995..50c34871410 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -147,17 +147,23 @@ public: current_keys_num = root_stat.numChildren - 1; } + std::vector>> exist_responses; for (const auto & [key, value] : new_values) { auto path = storage.fullPathForKey(key); - if (zookeeper->exists(path)) + exist_responses.push_back({&key, zookeeper->asyncExists(path)}); + } + + for (auto & [key, response] : exist_responses) + { + if (response.get().error == Coordination::Error::ZOK) { - requests.push_back(zkutil::makeSetRequest(path, value, -1)); + requests.push_back(zkutil::makeSetRequest(storage.fullPathForKey(*key), new_values[*key], -1)); } else { - requests.push_back(zkutil::makeCreateRequest(path, value, zkutil::CreateMode::Persistent)); + requests.push_back(zkutil::makeCreateRequest(storage.fullPathForKey(*key), new_values[*key], zkutil::CreateMode::Persistent)); ++new_keys_num; } } From c7d1dd942c5d51847da553512993c454af403a2c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Sep 2022 07:56:06 +0000 Subject: [PATCH 24/43] Address PR comments --- src/Common/Base64.cpp | 33 ++++++++++++++++++ src/Common/Base64.h | 10 ++++++ src/Coordination/KeeperStorage.cpp | 18 +++------- src/Storages/StorageKeeperMap.cpp | 55 ++++++++---------------------- 4 files changed, 63 insertions(+), 53 deletions(-) create mode 100644 src/Common/Base64.cpp create mode 100644 src/Common/Base64.h diff --git a/src/Common/Base64.cpp b/src/Common/Base64.cpp new file mode 100644 index 00000000000..74ce979b5b1 --- /dev/null +++ b/src/Common/Base64.cpp @@ -0,0 +1,33 @@ +#include + +#include +#include +#include +#include + +#include + +namespace DB +{ + +std::string base64Encode(const std::string & decoded, bool url_encoding) +{ + std::ostringstream ostr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + ostr.exceptions(std::ios::failbit); + Poco::Base64Encoder encoder(ostr, url_encoding ? Poco::BASE64_URL_ENCODING : 0); + encoder.rdbuf()->setLineLength(0); + encoder << decoded; + encoder.close(); + return ostr.str(); +} + +std::string base64Decode(const std::string & encoded, bool url_encoding) +{ + std::string decoded; + Poco::MemoryInputStream istr(encoded.data(), encoded.size()); + Poco::Base64Decoder decoder(istr, url_encoding ? Poco::BASE64_URL_ENCODING : 0); + Poco::StreamCopier::copyToString(decoder, decoded); + return decoded; +} + +} diff --git a/src/Common/Base64.h b/src/Common/Base64.h new file mode 100644 index 00000000000..39327cd5e08 --- /dev/null +++ b/src/Common/Base64.h @@ -0,0 +1,10 @@ +#pragma once + +namespace DB +{ + +std::string base64Encode(const std::string & decoded, bool url_encoding = false); + +std::string base64Decode(const std::string & encoded, bool url_encoding = false); + +} diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 693a1b16f0d..044afd34d5b 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1,11 +1,11 @@ #include #include -#include #include #include #include -#include #include + +#include #include #include #include @@ -15,8 +15,11 @@ #include #include #include + #include #include +#include + #include #include #include @@ -36,17 +39,6 @@ namespace ErrorCodes namespace { -String base64Encode(const String & decoded) -{ - std::ostringstream ostr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - ostr.exceptions(std::ios::failbit); - Poco::Base64Encoder encoder(ostr); - encoder.rdbuf()->setLineLength(0); - encoder << decoded; - encoder.close(); - return ostr.str(); -} - String getSHA1(const String & userdata) { Poco::SHA1Engine engine; diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 50c34871410..51886d6da3b 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -3,6 +3,7 @@ #include #include +#include #include @@ -23,20 +24,17 @@ #include #include -#include "Common/Exception.h" -#include "Common/ZooKeeper/IKeeper.h" +#include +#include +#include #include #include #include #include -#include "Core/UUID.h" -#include "base/types.h" + +#include #include -#include -#include -#include -#include namespace DB { @@ -53,33 +51,11 @@ namespace ErrorCodes namespace { -std::string base64Encode(const std::string & decoded) -{ - std::ostringstream ostr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - ostr.exceptions(std::ios::failbit); - Poco::Base64Encoder encoder(ostr, Poco::BASE64_URL_ENCODING); - encoder.rdbuf()->setLineLength(0); - encoder << decoded; - encoder.close(); - return ostr.str(); -} - -std::string base64Decode(const std::string & encoded) -{ - std::string decoded; - Poco::MemoryInputStream istr(encoded.data(), encoded.size()); - Poco::Base64Decoder decoder(istr, Poco::BASE64_URL_ENCODING); - Poco::StreamCopier::copyToString(decoder, decoded); - return decoded; -} - -constexpr std::string_view default_host = "default"; - std::string_view getBaseName(const std::string_view path) { auto last_slash = path.find_last_of('/'); if (last_slash == std::string_view::npos) - throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Failed to get basename of path '{}'", path); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to get basename of path '{}'", path); return path.substr(last_slash + 1); } @@ -98,7 +74,7 @@ public: { auto primary_key = storage.getPrimaryKey(); assert(primary_key.size() == 1); - primary_key_pos = getHeader().getPositionByName(storage.getPrimaryKey()[0]); + primary_key_pos = getHeader().getPositionByName(primary_key[0]); } std::string getName() const override { return "StorageKeeperMapSink"; } @@ -123,7 +99,7 @@ public: ++idx; } - auto key = base64Encode(wb_key.str()); + auto key = base64Encode(wb_key.str(), true); new_values[std::move(key)] = std::move(wb_value.str()); } } @@ -226,7 +202,7 @@ public: auto raw_keys = serializeKeysToRawString(it, end, key_column_type, max_block_size); for (auto & raw_key : raw_keys) - raw_key = base64Encode(raw_key); + raw_key = base64Encode(raw_key, true); return storage.getBySerializedKeys(raw_keys, nullptr); } @@ -257,7 +233,7 @@ StorageKeeperMap::StorageKeeperMap( , keys_limit(keys_limit_) , log(&Poco::Logger::get("StorageKeeperMap")) { - if (table_id.uuid == UUIDHelpers::Nil) + if (!table_id.hasUUID()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "KeeperMap cannot be used with '{}' database because UUID is needed. Please use Atomic or Replicated database", table_id.getDatabaseName()); setInMemoryMetadata(metadata); @@ -572,7 +548,7 @@ Chunk StorageKeeperMap::getByKeys(const ColumnsWithTypeAndName & keys, PaddedPOD auto raw_keys = serializeKeysToRawString(keys[0]); if (raw_keys.size() != keys[0].column->size()) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Assertion failed: {} != {}", raw_keys.size(), keys[0].column->size()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Assertion failed: {} != {}", raw_keys.size(), keys[0].column->size()); return getBySerializedKeys(raw_keys, &null_map); } @@ -622,7 +598,7 @@ Chunk StorageKeeperMap::getBySerializedKeys(const std::span k if (code == Coordination::Error::ZOK) { - fillColumns(base64Decode(keys[i]), response.data, primary_key_pos, sample_block, columns); + fillColumns(base64Decode(keys[i], true), response.data, primary_key_pos, sample_block, columns); } else if (code == Coordination::Error::ZNONODE) { @@ -658,11 +634,10 @@ StoragePtr create(const StorageFactory::Arguments & args) if (engine_args.empty() || engine_args.size() > 3) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage KeeperMap requires 1-2 arguments:\n" + "Storage KeeperMap requires 1-3 arguments:\n" "root_path: path in the Keeper where the values will be stored (required)\n" "create_missing_root_path: 1 if the root path should be created if it's missing, otherwise throw exception (default: 1)\n", - "keys_limit: number of keys allowed to be stored, 0 is no limit (default: 0)\n", - default_host); + "keys_limit: number of keys allowed to be stored, 0 is no limit (default: 0)"); auto root_path = checkAndGetLiteralArgument(engine_args[0], "root_path"); From e10acfb1fac34a95c18c83cc011c1d11a6dfd7a9 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Sep 2022 08:43:14 +0000 Subject: [PATCH 25/43] Fix stateless tests --- tests/queries/0_stateless/02416_keeper_map.sql | 2 +- tests/queries/0_stateless/02417_keeper_map_create_drop.sql | 2 +- tests/queries/0_stateless/02418_keeper_map_keys_limit.sql | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02416_keeper_map.sql b/tests/queries/0_stateless/02416_keeper_map.sql index 6688f2b4659..d4e6a80e66e 100644 --- a/tests/queries/0_stateless/02416_keeper_map.sql +++ b/tests/queries/0_stateless/02416_keeper_map.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest +-- Tags: no-ordinary-database, no-fasttest DROP TABLE IF EXISTS 02416_test SYNC; diff --git a/tests/queries/0_stateless/02417_keeper_map_create_drop.sql b/tests/queries/0_stateless/02417_keeper_map_create_drop.sql index fff1d78d589..9d6b67456d9 100644 --- a/tests/queries/0_stateless/02417_keeper_map_create_drop.sql +++ b/tests/queries/0_stateless/02417_keeper_map_create_drop.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest +-- Tags: no-ordinary-database, no-fasttest DROP TABLE IF EXISTS 02417_test SYNC; diff --git a/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql b/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql index aedeba661dd..59c13d83e60 100644 --- a/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql +++ b/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest +-- Tags: no-ordinary-database, no-fasttest DROP TABLE IF EXISTS 02418_test SYNC; From 6442824da872838222703124e7c004f3084ee70d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Sep 2022 10:47:05 +0000 Subject: [PATCH 26/43] Fix KeeperMap with Replicated database --- src/Storages/StorageKeeperMap.cpp | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 51886d6da3b..8bbf4ba3f8b 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -2,6 +2,8 @@ #include +#include + #include #include @@ -233,8 +235,9 @@ StorageKeeperMap::StorageKeeperMap( , keys_limit(keys_limit_) , log(&Poco::Logger::get("StorageKeeperMap")) { + auto database = DatabaseCatalog::instance().getDatabase(table_id.database_name); if (!table_id.hasUUID()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "KeeperMap cannot be used with '{}' database because UUID is needed. Please use Atomic or Replicated database", table_id.getDatabaseName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "KeeperMap cannot be used with '{}' database because it uses {} engine. Please use Atomic or Replicated database", table_id.getDatabaseName(), database->getEngineName()); setInMemoryMetadata(metadata); @@ -265,7 +268,13 @@ StorageKeeperMap::StorageKeeperMap( auto metadata_path_fs = root_path_fs / "ch_metadata"; metadata_path = metadata_path_fs; tables_path = metadata_path_fs / "tables"; - table_path = fs::path(tables_path) / toString(table_id.uuid); + + auto table_unique_id = toString(table_id.uuid); + if (const auto replicated_database = std::dynamic_pointer_cast(database)) + table_unique_id += replicated_database->getFullReplicaName(); + + table_path = fs::path(tables_path) / table_unique_id; + dropped_path = metadata_path_fs / "dropped"; dropped_lock_path = fs::path(dropped_path) / "lock"; From 62ff61404f3e2509dd8399cc28488f813212c33c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Sep 2022 12:16:13 +0000 Subject: [PATCH 27/43] Improve remove recursive --- src/Common/Base64.h | 2 ++ src/Common/ZooKeeper/ZooKeeper.cpp | 17 +++++++++++++---- src/Common/ZooKeeper/ZooKeeper.h | 19 ++++++++++++++++--- src/Interpreters/DDLWorker.cpp | 2 +- src/Storages/StorageKeeperMap.cpp | 4 ++-- 5 files changed, 34 insertions(+), 10 deletions(-) diff --git a/src/Common/Base64.h b/src/Common/Base64.h index 39327cd5e08..963d3acb48f 100644 --- a/src/Common/Base64.h +++ b/src/Common/Base64.h @@ -1,5 +1,7 @@ #pragma once +#include + namespace DB { diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index b95fa22ddb4..42d215318c9 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -720,7 +720,7 @@ void ZooKeeper::removeChildren(const std::string & path) } -void ZooKeeper::removeChildrenRecursive(const std::string & path, const std::string_view keep_child_node) +void ZooKeeper::removeChildrenRecursive(const std::string & path, RemoveException keep_child) { Strings children = getChildren(path); while (!children.empty()) @@ -728,18 +728,23 @@ void ZooKeeper::removeChildrenRecursive(const std::string & path, const std::str Coordination::Requests ops; for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i) { - if (likely(keep_child_node.empty() || keep_child_node != children.back())) + if (keep_child.path.empty() || keep_child.path != children.back()) [[likely]] { removeChildrenRecursive(fs::path(path) / children.back()); ops.emplace_back(makeRemoveRequest(fs::path(path) / children.back(), -1)); } + else if (keep_child.remove_subtree) + { + removeChildrenRecursive(fs::path(path) / children.back()); + } + children.pop_back(); } multi(ops); } } -bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probably_flat, const std::string_view keep_child_node) +bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probably_flat, RemoveException keep_child) { Strings children; if (tryGetChildren(path, children) != Coordination::Error::ZOK) @@ -756,7 +761,7 @@ bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probab { String child_path = fs::path(path) / children.back(); - if (likely(keep_child_node.empty() || keep_child_node != children.back())) + if (keep_child.path.empty() || keep_child.path != children.back()) [[likely]] { /// Will try to avoid recursive getChildren calls if child_path probably has no children. /// It may be extremely slow when path contain a lot of leaf children. @@ -766,6 +771,10 @@ bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probab batch.push_back(child_path); ops.emplace_back(zkutil::makeRemoveRequest(child_path, -1)); } + else if (keep_child.remove_subtree && !probably_flat) + { + tryRemoveChildrenRecursive(child_path); + } children.pop_back(); } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index d16f1f96271..65bede5abe2 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -58,6 +58,18 @@ struct ShuffleHost } }; +struct RemoveException +{ + explicit RemoveException(std::string_view path_ = "", bool remove_subtree_ = true) + : path(path_) + , remove_subtree(remove_subtree_) + {} + + std::string_view path; + // whether we should keep the child node and its subtree or just the child node + bool remove_subtree; +}; + using GetPriorityForLoadBalancing = DB::GetPriorityForLoadBalancing; /// ZooKeeper session. The interface is substantially different from the usual libzookeeper API. @@ -232,12 +244,13 @@ public: void tryRemoveRecursive(const std::string & path); /// Similar to removeRecursive(...) and tryRemoveRecursive(...), but does not remove path itself. - /// If keep_child_node is not empty, this method will not remove path/keep_child_node and its subtree. - void removeChildrenRecursive(const std::string & path, std::string_view keep_child_node = {}); + /// Node defined as RemoveException will not be deleted. + void removeChildrenRecursive(const std::string & path, RemoveException keep_child = RemoveException{}); /// If probably_flat is true, this method will optimistically try to remove children non-recursive /// and will fall back to recursive removal if it gets ZNOTEMPTY for some child. /// Returns true if no kind of fallback happened. - bool tryRemoveChildrenRecursive(const std::string & path, bool probably_flat = false, std::string_view keep_child_node = {}); + /// Node defined as RemoveException will not be deleted. + bool tryRemoveChildrenRecursive(const std::string & path, bool probably_flat = false, RemoveException keep_child= RemoveException{}); /// Remove all children nodes (non recursive). void removeChildren(const std::string & path); diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 6ec20ab5f5f..c8878297c02 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -890,7 +890,7 @@ void DDLWorker::cleanupQueue(Int64, const ZooKeeperPtr & zookeeper) /// We recursively delete all nodes except node_path/finished to prevent staled hosts from /// creating node_path/active node (see createStatusDirs(...)) - zookeeper->tryRemoveChildrenRecursive(node_path, /* probably_flat */ false, "finished"); + zookeeper->tryRemoveChildrenRecursive(node_path, /* probably_flat */ false, zkutil::RemoveException{"finished"}); /// And then we remove node_path and node_path/finished in a single transaction Coordination::Requests ops; diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 8bbf4ba3f8b..85ce8931ea7 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -411,12 +411,12 @@ void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont { checkTable(); auto client = getClient(); - client->tryRemoveChildrenRecursive(root_path, true, getBaseName(metadata_path)); + client->tryRemoveChildrenRecursive(root_path, true, zkutil::RemoveException{getBaseName(metadata_path), /*remove_subtree_*/ false}); } bool StorageKeeperMap::dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock) { - zookeeper->removeChildrenRecursive(root_path, getBaseName(metadata_path)); + zookeeper->removeChildrenRecursive(root_path, zkutil::RemoveException{getBaseName(metadata_path), /*remove_subtree_*/ false}); bool completely_removed = false; Coordination::Requests ops; From dbf67f26d8923a20d5bfd1ff24842a350ef60745 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Sep 2022 14:39:57 +0000 Subject: [PATCH 28/43] Add prefix path --- src/Storages/StorageKeeperMap.cpp | 16 +++++++++++----- src/Storages/StorageKeeperMap.h | 2 +- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 85ce8931ea7..627ae4d4a93 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -235,6 +235,10 @@ StorageKeeperMap::StorageKeeperMap( , keys_limit(keys_limit_) , log(&Poco::Logger::get("StorageKeeperMap")) { + std::string path_prefix = context_->getConfigRef().getString("keeper_map_path_prefix", ""); + if (path_prefix.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "KeeperMap is disabled because 'keeper_map_path_prefix' config is not defined"); + auto database = DatabaseCatalog::instance().getDatabase(table_id.database_name); if (!table_id.hasUUID()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "KeeperMap cannot be used with '{}' database because it uses {} engine. Please use Atomic or Replicated database", table_id.getDatabaseName(), database->getEngineName()); @@ -246,7 +250,6 @@ StorageKeeperMap::StorageKeeperMap( if (!root_path.starts_with('/')) throw Exception("root_path should start with '/'", ErrorCodes::BAD_ARGUMENTS); - auto config_keys_limit = context_->getConfigRef().getUInt64("keeper_map_keys_limit", 0); if (config_keys_limit != 0 && keys_limit > config_keys_limit) { @@ -264,7 +267,9 @@ StorageKeeperMap::StorageKeeperMap( LOG_INFO(log, "Keys limit for {} will be set to {}", getStorageID().getFullTableName(), keys_limit); } - std::filesystem::path root_path_fs{root_path}; + auto root_path_fs = fs::path(path_prefix) / std::string_view{root_path}.substr(1); + root_path = root_path_fs.generic_string(); + auto metadata_path_fs = root_path_fs / "ch_metadata"; metadata_path = metadata_path_fs; tables_path = metadata_path_fs / "tables"; @@ -286,7 +291,7 @@ StorageKeeperMap::StorageKeeperMap( auto client = getClient(); - if (root_path != "/" && !client->exists(root_path)) + if (root_path_fs != "/" && !client->exists(root_path)) { if (!create_missing_root_path) { @@ -312,7 +317,7 @@ StorageKeeperMap::StorageKeeperMap( if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) { - LOG_INFO(log, "Someone else removed leftovers"); + LOG_INFO(log, "Someone else removed leftover nodes"); } else if (code != Coordination::Error::ZOK) { @@ -648,7 +653,8 @@ StoragePtr create(const StorageFactory::Arguments & args) "create_missing_root_path: 1 if the root path should be created if it's missing, otherwise throw exception (default: 1)\n", "keys_limit: number of keys allowed to be stored, 0 is no limit (default: 0)"); - auto root_path = checkAndGetLiteralArgument(engine_args[0], "root_path"); + const auto root_path_node = evaluateConstantExpressionAsLiteral(engine_args[0], args.getLocalContext()); + auto root_path = checkAndGetLiteralArgument(root_path_node, "root_path"); bool create_missing_root_path = true; if (engine_args.size() > 1) diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 3e47e5ee9c6..03360dd31dd 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -105,7 +105,7 @@ private: std::optional isTableValid() const; - const std::string root_path; + std::string root_path; std::string primary_key; std::string metadata_path; From c91369ec5a6c59e3098252e6c51251856d7ec4c2 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Sep 2022 07:24:46 +0000 Subject: [PATCH 29/43] Use database in path to support parallel --- tests/queries/0_stateless/02416_keeper_map.sql | 12 ++++++------ .../0_stateless/02417_keeper_map_create_drop.sql | 4 ++-- .../0_stateless/02418_keeper_map_keys_limit.sql | 4 ++-- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02416_keeper_map.sql b/tests/queries/0_stateless/02416_keeper_map.sql index d4e6a80e66e..caf44fc0ed6 100644 --- a/tests/queries/0_stateless/02416_keeper_map.sql +++ b/tests/queries/0_stateless/02416_keeper_map.sql @@ -2,13 +2,13 @@ DROP TABLE IF EXISTS 02416_test SYNC; -CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/test2381'); -- { serverError 36 } -CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key2); -- { serverError 47 } -CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key, value); -- { serverError 36 } -CREATE TABLE 02416_test (key Tuple(String, UInt32), value UInt64) Engine=KeeperMap('/test2381') PRIMARY KEY(key); +CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416'); -- { serverError 36 } +CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(key2); -- { serverError 47 } +CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(key, value); -- { serverError 36 } +CREATE TABLE 02416_test (key Tuple(String, UInt32), value UInt64) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(key); DROP TABLE IF EXISTS 02416_test SYNC; -CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/test2381') PRIMARY KEY(key); +CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(key); INSERT INTO 02416_test SELECT '1_1', number FROM numbers(10000); SELECT COUNT(1) == 1 FROM 02416_test; @@ -21,7 +21,7 @@ SELECT SUM(value) == 1 + 99 + 900 FROM 02416_test WHERE key IN ('1_1', '99_1', ' DROP TABLE IF EXISTS 02416_test SYNC; DROP TABLE IF EXISTS 02416_test_memory; -CREATE TABLE 02416_test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=KeeperMap('/test2381') PRIMARY KEY(k); +CREATE TABLE 02416_test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(k); CREATE TABLE 02416_test_memory AS 02416_test Engine = Memory; INSERT INTO 02416_test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; diff --git a/tests/queries/0_stateless/02417_keeper_map_create_drop.sql b/tests/queries/0_stateless/02417_keeper_map_create_drop.sql index 9d6b67456d9..49340167eaa 100644 --- a/tests/queries/0_stateless/02417_keeper_map_create_drop.sql +++ b/tests/queries/0_stateless/02417_keeper_map_create_drop.sql @@ -2,12 +2,12 @@ DROP TABLE IF EXISTS 02417_test SYNC; -CREATE TABLE 02417_test (key UInt64, value UInt64) Engine=KeeperMap('/test2417') PRIMARY KEY(key); +CREATE TABLE 02417_test (key UInt64, value UInt64) Engine=KeeperMap('/' || currentDatabase() || '/test2417') PRIMARY KEY(key); INSERT INTO 02417_test VALUES (1, 11); SELECT * FROM 02417_test ORDER BY key; SELECT '------'; -CREATE TABLE 02417_test_another (key UInt64, value UInt64) Engine=KeeperMap('/test2417') PRIMARY KEY(key); +CREATE TABLE 02417_test_another (key UInt64, value UInt64) Engine=KeeperMap('/' || currentDatabase() || '/test2417') PRIMARY KEY(key); INSERT INTO 02417_test_another VALUES (2, 22); SELECT * FROM 02417_test_another ORDER BY key; SELECT '------'; diff --git a/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql b/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql index 59c13d83e60..de84bde3aaf 100644 --- a/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql +++ b/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS 02418_test SYNC; -CREATE TABLE 02418_test (key UInt64, value Float64) Engine=KeeperMap('/test2418', 1, 3) PRIMARY KEY(key); +CREATE TABLE 02418_test (key UInt64, value Float64) Engine=KeeperMap('/' || currentDatabase() || '/test2418', 1, 3) PRIMARY KEY(key); INSERT INTO 02418_test VALUES (1, 1.1), (2, 2.2); SELECT count() FROM 02418_test; @@ -12,7 +12,7 @@ INSERT INTO 02418_test VALUES (3, 3.3), (4, 4.4); -- { serverError 290 } INSERT INTO 02418_test VALUES (1, 2.1), (2, 3.2), (3, 3.3); SELECT count() FROM 02418_test; -CREATE TABLE 02418_test_another (key UInt64, value Float64) Engine=KeeperMap('/test2418', 1, 4) PRIMARY KEY(key); +CREATE TABLE 02418_test_another (key UInt64, value Float64) Engine=KeeperMap('/' || currentDatabase() || '/test2418', 1, 4) PRIMARY KEY(key); INSERT INTO 02418_test VALUES (4, 4.4); -- { serverError 290 } INSERT INTO 02418_test_another VALUES (4, 4.4); From b02bf49939f61ad6fed8006e80a7be6802ecb29c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Sep 2022 07:34:26 +0000 Subject: [PATCH 30/43] Use path prefix --- .../test_keeper_map/configs/enable_keeper_map.xml | 3 +++ tests/integration/test_keeper_map/test.py | 11 +++++------ 2 files changed, 8 insertions(+), 6 deletions(-) create mode 100644 tests/integration/test_keeper_map/configs/enable_keeper_map.xml diff --git a/tests/integration/test_keeper_map/configs/enable_keeper_map.xml b/tests/integration/test_keeper_map/configs/enable_keeper_map.xml new file mode 100644 index 00000000000..b4cbb6a954b --- /dev/null +++ b/tests/integration/test_keeper_map/configs/enable_keeper_map.xml @@ -0,0 +1,3 @@ + + /test_keeper_map + diff --git a/tests/integration/test_keeper_map/test.py b/tests/integration/test_keeper_map/test.py index 025439f44db..8a4ae701599 100644 --- a/tests/integration/test_keeper_map/test.py +++ b/tests/integration/test_keeper_map/test.py @@ -17,7 +17,7 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", - main_configs=[], + main_configs=['configs/enable_keeper_map.xml'], with_zookeeper=True, stay_alive=True, ) @@ -34,7 +34,6 @@ def started_cluster(): def get_genuine_zk(): - print("Zoo1", cluster.get_instance_ip("zoo1")) return cluster.get_kazoo_client("zoo1") @@ -58,7 +57,7 @@ def test_create_keeper_map(started_cluster): zk_client = get_genuine_zk() def assert_children_size(expected_size): - assert len(zk_client.get_children("/test1")) == expected_size + assert len(zk_client.get_children("/test_keeper_map/test1")) == expected_size assert_children_size(1) @@ -91,7 +90,6 @@ def create_drop_loop(index, stop_event): if stop_event.is_set(): return - stdout.write(f"Trying with {i} for {index}\n") node.query( f"CREATE TABLE {table_name} (key UInt64, value UInt64) ENGINE = KeeperMap('/test') PRIMARY KEY(key);" ) @@ -127,7 +125,8 @@ def test_create_drop_keeper_map_concurrent(started_cluster): pool.close() client = get_genuine_zk() - assert len(client.get_children("/test")) == 0 + assert len(client.get_children("/test_keeper_map/test")) == 0 + client.stop() def test_keeper_map_without_zk(started_cluster): @@ -162,7 +161,7 @@ def test_keeper_map_without_zk(started_cluster): node.query("SELECT * FROM test_keeper_map") client = get_genuine_zk() - remove_children(client, "/test1") + remove_children(client, "/test_keeper_map/test1") node.restart_clickhouse(60) error = node.query_and_get_error("SELECT * FROM test_keeper_map") assert "Failed to activate table because of invalid metadata in ZooKeeper" in error From 1a48beec30f6947de1d8a3728c83cde2125a43d6 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Sep 2022 07:40:39 +0000 Subject: [PATCH 31/43] Use server UUID --- src/Storages/StorageKeeperMap.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 627ae4d4a93..132e745ac8d 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -6,6 +6,7 @@ #include #include +#include #include @@ -274,10 +275,7 @@ StorageKeeperMap::StorageKeeperMap( metadata_path = metadata_path_fs; tables_path = metadata_path_fs / "tables"; - auto table_unique_id = toString(table_id.uuid); - if (const auto replicated_database = std::dynamic_pointer_cast(database)) - table_unique_id += replicated_database->getFullReplicaName(); - + auto table_unique_id = toString(table_id.uuid) + toString(ServerUUID::get()); table_path = fs::path(tables_path) / table_unique_id; dropped_path = metadata_path_fs / "dropped"; From 41b431c8b4d423ad0040963158b2be269d5592ed Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Sep 2022 07:41:25 +0000 Subject: [PATCH 32/43] Enable keeper_map in tests --- tests/config/config.d/enable_keeper_map.xml | 3 +++ 1 file changed, 3 insertions(+) create mode 100644 tests/config/config.d/enable_keeper_map.xml diff --git a/tests/config/config.d/enable_keeper_map.xml b/tests/config/config.d/enable_keeper_map.xml new file mode 100644 index 00000000000..b4cbb6a954b --- /dev/null +++ b/tests/config/config.d/enable_keeper_map.xml @@ -0,0 +1,3 @@ + + /test_keeper_map + From f5fc3db0dd0da8299d21d591dbe005ea0ed786f7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Sep 2022 07:42:22 +0000 Subject: [PATCH 33/43] Small fix --- src/Storages/StorageKeeperMap.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 132e745ac8d..8f6f16a7098 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -289,7 +289,7 @@ StorageKeeperMap::StorageKeeperMap( auto client = getClient(); - if (root_path_fs != "/" && !client->exists(root_path)) + if (root_path != "/" && !client->exists(root_path)) { if (!create_missing_root_path) { From 46f74fcbaffea708f8990b912da7f286e9d89901 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 2 Sep 2022 07:49:09 +0000 Subject: [PATCH 34/43] Automatic style fix --- tests/integration/test_keeper_map/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_keeper_map/test.py b/tests/integration/test_keeper_map/test.py index 8a4ae701599..7dce77dec42 100644 --- a/tests/integration/test_keeper_map/test.py +++ b/tests/integration/test_keeper_map/test.py @@ -17,7 +17,7 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", - main_configs=['configs/enable_keeper_map.xml'], + main_configs=["configs/enable_keeper_map.xml"], with_zookeeper=True, stay_alive=True, ) From a326450ddf506d1e25913320c4a6a98f9465cef6 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Sep 2022 13:12:02 +0000 Subject: [PATCH 35/43] Install enable_keeper_map config --- tests/config/install.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/config/install.sh b/tests/config/install.sh index e7d0f8e7acf..91be5286102 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -50,6 +50,7 @@ ln -sf $SRC_PATH/config.d/filesystem_cache_log.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/session_log.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/enable_zero_copy_replication.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/nlp.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/enable_keeper_map.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/users.d/log_queries.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/readonly.xml $DEST_SERVER_PATH/users.d/ From dc17a125f8cfe7206ecb4448a59660640a8eb964 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 5 Sep 2022 07:00:52 +0000 Subject: [PATCH 36/43] No parallel for heavy KeeperMap test --- tests/queries/0_stateless/02416_keeper_map.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02416_keeper_map.sql b/tests/queries/0_stateless/02416_keeper_map.sql index caf44fc0ed6..bde07db1328 100644 --- a/tests/queries/0_stateless/02416_keeper_map.sql +++ b/tests/queries/0_stateless/02416_keeper_map.sql @@ -1,4 +1,4 @@ --- Tags: no-ordinary-database, no-fasttest +-- Tags: no-ordinary-database, no-fasttest, no-parallel DROP TABLE IF EXISTS 02416_test SYNC; From 569f0abfb7238439c454c44378d48c2736b188c9 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 5 Sep 2022 07:03:23 +0000 Subject: [PATCH 37/43] Change error code --- src/Storages/StorageKeeperMap.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 8f6f16a7098..ea333c44362 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -603,7 +603,7 @@ Chunk StorageKeeperMap::getBySerializedKeys(const std::span k continue; if (value.wait_until(wait_until) != std::future_status::ready) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Failed to fetch values: timeout"); + throw DB::Exception(ErrorCodes::KEEPER_EXCEPTION, "Failed to fetch values: timeout"); auto response = value.get(); Coordination::Error code = response.error; From 4d42597b789dc548729b3decdddd4e9dc8611d2a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 5 Sep 2022 10:09:52 +0000 Subject: [PATCH 38/43] Make test less heavy --- tests/queries/0_stateless/02416_keeper_map.sql | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02416_keeper_map.sql b/tests/queries/0_stateless/02416_keeper_map.sql index bde07db1328..3866c33f03d 100644 --- a/tests/queries/0_stateless/02416_keeper_map.sql +++ b/tests/queries/0_stateless/02416_keeper_map.sql @@ -10,11 +10,11 @@ CREATE TABLE 02416_test (key Tuple(String, UInt32), value UInt64) Engine=KeeperM DROP TABLE IF EXISTS 02416_test SYNC; CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(key); -INSERT INTO 02416_test SELECT '1_1', number FROM numbers(10000); +INSERT INTO 02416_test SELECT '1_1', number FROM numbers(1000); SELECT COUNT(1) == 1 FROM 02416_test; -INSERT INTO 02416_test SELECT concat(toString(number), '_1'), number FROM numbers(10000); -SELECT COUNT(1) == 10000 FROM 02416_test; +INSERT INTO 02416_test SELECT concat(toString(number), '_1'), number FROM numbers(1000); +SELECT COUNT(1) == 1000 FROM 02416_test; SELECT uniqExact(key) == 32 FROM (SELECT * FROM 02416_test LIMIT 32 SETTINGS max_block_size = 1); SELECT SUM(value) == 1 + 99 + 900 FROM 02416_test WHERE key IN ('1_1', '99_1', '900_1'); @@ -24,9 +24,9 @@ DROP TABLE IF EXISTS 02416_test_memory; CREATE TABLE 02416_test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(k); CREATE TABLE 02416_test_memory AS 02416_test Engine = Memory; -INSERT INTO 02416_test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; +INSERT INTO 02416_test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000) group by k; -INSERT INTO 02416_test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; +INSERT INTO 02416_test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000) group by k; SELECT A.a = B.a, A.b = B.b, A.c = B.c, A.d = B.d, A.e = B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 02416_test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 02416_test_memory) B USING a ORDER BY a; From b99996961ddcb3126dae15692e21a5710c692f54 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 8 Sep 2022 12:45:36 +0000 Subject: [PATCH 39/43] Address PR comments --- src/Storages/StorageKeeperMap.cpp | 201 ++++++++++++------ src/Storages/StorageKeeperMap.h | 6 +- tests/integration/test_keeper_map/test.py | 4 - .../queries/0_stateless/02416_keeper_map.sql | 3 +- .../02418_keeper_map_keys_limit.sql | 4 +- .../02419_keeper_map_primary_key.reference | 6 + .../02419_keeper_map_primary_key.sh | 23 ++ 7 files changed, 174 insertions(+), 73 deletions(-) create mode 100644 tests/queries/0_stateless/02419_keeper_map_primary_key.reference create mode 100755 tests/queries/0_stateless/02419_keeper_map_primary_key.sh diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index ea333c44362..de7456d0df1 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include @@ -63,6 +64,29 @@ std::string_view getBaseName(const std::string_view path) return path.substr(last_slash + 1); } +std::string formattedAST(const ASTPtr & ast) +{ + if (!ast) + return ""; + WriteBufferFromOwnString buf; + formatAST(*ast, buf, false, true); + return buf.str(); +} + +void verifyTableId(const StorageID & table_id) +{ + if (!table_id.hasUUID()) + { + auto database = DatabaseCatalog::instance().getDatabase(table_id.database_name); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "KeeperMap cannot be used with '{}' database because it uses {} engine. Please use Atomic or Replicated database", + table_id.getDatabaseName(), + database->getEngineName()); + } + +} + } class StorageKeeperMapSink : public SinkToStorage @@ -118,6 +142,8 @@ public: size_t current_keys_num = 0; size_t new_keys_num = 0; + // We use keys limit as a soft limit so we ignore some cases when it can be still exceeded + // (e.g if parallel insert queries are being run) if (keys_limit != 0) { Coordination::Stat root_stat; @@ -226,7 +252,6 @@ StorageKeeperMap::StorageKeeperMap( bool attach, std::string_view primary_key_, const std::string & root_path_, - bool create_missing_root_path, UInt64 keys_limit_) : IStorage(table_id) , WithContext(context_->getGlobalContext()) @@ -234,44 +259,47 @@ StorageKeeperMap::StorageKeeperMap( , primary_key(primary_key_) , zookeeper_name(zkutil::extractZooKeeperName(root_path_)) , keys_limit(keys_limit_) - , log(&Poco::Logger::get("StorageKeeperMap")) + , log(&Poco::Logger::get(fmt::format("StorageKeeperMap ({})", table_id.getNameForLogs()))) { std::string path_prefix = context_->getConfigRef().getString("keeper_map_path_prefix", ""); if (path_prefix.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "KeeperMap is disabled because 'keeper_map_path_prefix' config is not defined"); - auto database = DatabaseCatalog::instance().getDatabase(table_id.database_name); - if (!table_id.hasUUID()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "KeeperMap cannot be used with '{}' database because it uses {} engine. Please use Atomic or Replicated database", table_id.getDatabaseName(), database->getEngineName()); + verifyTableId(table_id); setInMemoryMetadata(metadata); + WriteBufferFromOwnString out; + out << "KeeperMap metadata format version: 1\n" + << "columns: " << metadata.columns.toString() + << "primary key: " << formattedAST(metadata.getPrimaryKey().expression_list_ast) << "\n"; + metadata_string = out.str(); + if (root_path.empty()) throw Exception("root_path should not be empty", ErrorCodes::BAD_ARGUMENTS); if (!root_path.starts_with('/')) throw Exception("root_path should start with '/'", ErrorCodes::BAD_ARGUMENTS); auto config_keys_limit = context_->getConfigRef().getUInt64("keeper_map_keys_limit", 0); - if (config_keys_limit != 0 && keys_limit > config_keys_limit) + if (config_keys_limit != 0 && (keys_limit == 0 || keys_limit > config_keys_limit)) { LOG_WARNING( log, - "Keys limit for {} defined by argument ({}) is larger than the one defined by 'keeper_map_keys_limit' config ({}). Will use " + "Keys limit defined by argument ({}) is larger than the one defined by 'keeper_map_keys_limit' config ({}). Will use " "config defined value", - getStorageID().getFullTableName(), keys_limit, config_keys_limit); keys_limit = config_keys_limit; } else if (keys_limit > 0) { - LOG_INFO(log, "Keys limit for {} will be set to {}", getStorageID().getFullTableName(), keys_limit); + LOG_INFO(log, "Keys limit will be set to {}", keys_limit); } auto root_path_fs = fs::path(path_prefix) / std::string_view{root_path}.substr(1); root_path = root_path_fs.generic_string(); - auto metadata_path_fs = root_path_fs / "ch_metadata"; + auto metadata_path_fs = root_path_fs / "metadata"; metadata_path = metadata_path_fs; tables_path = metadata_path_fs / "tables"; @@ -291,19 +319,9 @@ StorageKeeperMap::StorageKeeperMap( if (root_path != "/" && !client->exists(root_path)) { - if (!create_missing_root_path) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Path '{}' doesn't exist. Please create it or set 'create_missing_root_path' to true'", - root_path_); - } - else - { - LOG_TRACE(log, "Creating root path {}", root_path); - client->createAncestors(root_path); - client->createIfNotExists(root_path, ""); - } + LOG_TRACE(log, "Creating root path {}", root_path); + client->createAncestors(root_path); + client->createIfNotExists(root_path, ""); } for (size_t i = 0; i < 1000; ++i) @@ -313,10 +331,15 @@ StorageKeeperMap::StorageKeeperMap( LOG_INFO(log, "Removing leftover nodes"); auto code = client->tryCreate(dropped_lock_path, "", zkutil::CreateMode::Ephemeral); - if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) + if (code == Coordination::Error::ZNONODE) { LOG_INFO(log, "Someone else removed leftover nodes"); } + else if (code == Coordination::Error::ZNODEEXISTS) + { + LOG_INFO(log, "Someone else is removing leftover nodes"); + continue; + } else if (code != Coordination::Error::ZOK) { throw Coordination::Exception(code, dropped_lock_path); @@ -329,7 +352,29 @@ StorageKeeperMap::StorageKeeperMap( } } - client->createIfNotExists(metadata_path, ""); + std::string stored_metadata_string; + auto exists = client->tryGet(metadata_path, stored_metadata_string); + + if (exists) + { + // this requires same name for columns + // maybe we can do a smarter comparison for columns and primary key expression + if (stored_metadata_string != metadata_string) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Path {} is already used but the stored table definition doesn't match. Stored metadata: {}", + root_path, + stored_metadata_string); + } + else + { + auto code = client->tryCreate(metadata_path, metadata_string, zkutil::CreateMode::Persistent); + if (code == Coordination::Error::ZNODEEXISTS) + continue; + else if (code != Coordination::Error::ZOK) + throw Coordination::Exception(code, metadata_path); + } + client->createIfNotExists(tables_path, ""); auto code = client->tryCreate(table_path, "", zkutil::CreateMode::Persistent); @@ -399,7 +444,15 @@ Pipe StorageKeeperMap::read( auto client = getClient(); if (all_scan) - return process_keys(std::make_shared>(client->getChildren(root_path))); + { + auto children = std::make_shared>(client->getChildren(root_path)); + std::erase_if(*children, [&](const std::string_view key) + { + return fullPathForKey(key) == metadata_path; + }); + + return process_keys(std::move(children)); + } return process_keys(std::move(filtered_keys)); } @@ -525,29 +578,51 @@ std::optional StorageKeeperMap::isTableValid() const if (table_is_valid.has_value()) return *table_is_valid; - try + [&] { - // validate all metadata nodes are present - Coordination::Requests requests; - requests.push_back(zkutil::makeCheckRequest(table_path, -1)); + try + { + auto client = getClient(); - Coordination::Responses responses; - auto client = getClient(); - auto res = client->tryMulti(requests, responses); - table_is_valid = res == Coordination::Error::ZOK; - } - catch (const Coordination::Exception & e) - { - tryLogCurrentException(log); + std::string stored_metadata_string; + Coordination::Stat metadata_stat; + client->tryGet(metadata_path, stored_metadata_string, &metadata_stat); - if (!Coordination::isHardwareError(e.code)) - table_is_valid = false; - } - catch (const Exception &) - { - tryLogCurrentException(log); - table_is_valid = false; - } + if (metadata_stat.numChildren == 0) + { + table_is_valid = false; + return; + } + + if (metadata_string != stored_metadata_string) + { + LOG_ERROR( + log, + "Table definition does not match to the one stored in the path {}. Stored defintion: {}", + root_path, + stored_metadata_string); + table_is_valid = false; + return; + } + + // validate all metadata nodes are present + Coordination::Requests requests; + requests.push_back(zkutil::makeCheckRequest(table_path, -1)); + requests.push_back(zkutil::makeCheckRequest(dropped_path, -1)); + + Coordination::Responses responses; + client->tryMulti(requests, responses); + + table_is_valid = responses[0]->error == Coordination::Error::ZOK && responses[1]->error == Coordination::Error::ZNONODE; + } + catch (const Coordination::Exception & e) + { + tryLogCurrentException(log); + + if (!Coordination::isHardwareError(e.code)) + table_is_valid = false; + } + }(); return table_is_valid; } @@ -585,12 +660,6 @@ Chunk StorageKeeperMap::getBySerializedKeys(const std::span k for (const auto & key : keys) { const auto full_path = fullPathForKey(key); - if (full_path == metadata_path) - { - values.emplace_back(); - continue; - } - values.emplace_back(client->asyncTryGet(full_path)); } @@ -599,9 +668,6 @@ Chunk StorageKeeperMap::getBySerializedKeys(const std::span k for (size_t i = 0; i < keys.size(); ++i) { auto & value = values[i]; - if (!value.valid()) - continue; - if (value.wait_until(wait_until) != std::future_status::ready) throw DB::Exception(ErrorCodes::KEEPER_EXCEPTION, "Failed to fetch values: timeout"); @@ -634,7 +700,17 @@ Chunk StorageKeeperMap::getBySerializedKeys(const std::span k Block StorageKeeperMap::getSampleBlock(const Names &) const { auto metadata = getInMemoryMetadataPtr(); - return metadata ? metadata->getSampleBlock() : Block(); + return metadata->getSampleBlock(); +} + +void StorageKeeperMap::checkTableCanBeRenamed(const StorageID & new_name) const +{ + verifyTableId(new_name); +} + +void StorageKeeperMap::rename(const String & /*new_path_to_table_data*/, const StorageID & new_table_id) +{ + checkTableCanBeRenamed(new_table_id); } namespace @@ -643,24 +719,19 @@ namespace StoragePtr create(const StorageFactory::Arguments & args) { ASTs & engine_args = args.engine_args; - if (engine_args.empty() || engine_args.size() > 3) + if (engine_args.empty() || engine_args.size() > 2) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage KeeperMap requires 1-3 arguments:\n" "root_path: path in the Keeper where the values will be stored (required)\n" - "create_missing_root_path: 1 if the root path should be created if it's missing, otherwise throw exception (default: 1)\n", "keys_limit: number of keys allowed to be stored, 0 is no limit (default: 0)"); const auto root_path_node = evaluateConstantExpressionAsLiteral(engine_args[0], args.getLocalContext()); auto root_path = checkAndGetLiteralArgument(root_path_node, "root_path"); - bool create_missing_root_path = true; - if (engine_args.size() > 1) - create_missing_root_path = checkAndGetLiteralArgument(engine_args[1], "create_missing_root_path"); - UInt64 keys_limit = 0; - if (engine_args.size() > 2) - keys_limit = checkAndGetLiteralArgument(engine_args[2], "keys_limit"); + if (engine_args.size() > 1) + keys_limit = checkAndGetLiteralArgument(engine_args[1], "keys_limit"); StorageInMemoryMetadata metadata; metadata.setColumns(args.columns); @@ -675,7 +746,7 @@ StoragePtr create(const StorageFactory::Arguments & args) throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS); return std::make_shared( - args.getContext(), args.table_id, metadata, args.query.attach, primary_key_names[0], root_path, create_missing_root_path, keys_limit); + args.getContext(), args.table_id, metadata, args.query.attach, primary_key_names[0], root_path, keys_limit); } } diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 03360dd31dd..504d3df087d 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -30,7 +30,6 @@ public: bool attach, std::string_view primary_key_, const std::string & root_path_, - bool create_missing_root_path, UInt64 keys_limit_); Pipe read( @@ -55,6 +54,9 @@ public: Block getSampleBlock(const Names &) const override; + void checkTableCanBeRenamed(const StorageID & new_name) const override; + void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; + bool supportsParallelInsert() const override { return true; } bool supportsIndexForIn() const override { return true; } bool mayBenefitFromIndexForIn( @@ -118,6 +120,8 @@ private: std::string zookeeper_name; + std::string metadata_string; + uint64_t keys_limit{0}; mutable std::mutex zookeeper_mutex; diff --git a/tests/integration/test_keeper_map/test.py b/tests/integration/test_keeper_map/test.py index 7dce77dec42..7801dd3c57f 100644 --- a/tests/integration/test_keeper_map/test.py +++ b/tests/integration/test_keeper_map/test.py @@ -47,10 +47,6 @@ def remove_children(client, path): def test_create_keeper_map(started_cluster): - assert "Path '/test1' doesn't exist" in node.query_and_get_error( - "CREATE TABLE test_keeper_map (key UInt64, value UInt64) ENGINE = KeeperMap('/test1', 0) PRIMARY KEY(key);" - ) - node.query( "CREATE TABLE test_keeper_map (key UInt64, value UInt64) ENGINE = KeeperMap('/test1') PRIMARY KEY(key);" ) diff --git a/tests/queries/0_stateless/02416_keeper_map.sql b/tests/queries/0_stateless/02416_keeper_map.sql index 3866c33f03d..a87ad6d914b 100644 --- a/tests/queries/0_stateless/02416_keeper_map.sql +++ b/tests/queries/0_stateless/02416_keeper_map.sql @@ -1,10 +1,11 @@ --- Tags: no-ordinary-database, no-fasttest, no-parallel +-- Tags: no-ordinary-database, no-fasttest DROP TABLE IF EXISTS 02416_test SYNC; CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416'); -- { serverError 36 } CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(key2); -- { serverError 47 } CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(key, value); -- { serverError 36 } +CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(concat(key, value)); -- { serverError 36 } CREATE TABLE 02416_test (key Tuple(String, UInt32), value UInt64) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(key); DROP TABLE IF EXISTS 02416_test SYNC; diff --git a/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql b/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql index de84bde3aaf..3d2055b85ea 100644 --- a/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql +++ b/tests/queries/0_stateless/02418_keeper_map_keys_limit.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS 02418_test SYNC; -CREATE TABLE 02418_test (key UInt64, value Float64) Engine=KeeperMap('/' || currentDatabase() || '/test2418', 1, 3) PRIMARY KEY(key); +CREATE TABLE 02418_test (key UInt64, value Float64) Engine=KeeperMap('/' || currentDatabase() || '/test2418', 3) PRIMARY KEY(key); INSERT INTO 02418_test VALUES (1, 1.1), (2, 2.2); SELECT count() FROM 02418_test; @@ -12,7 +12,7 @@ INSERT INTO 02418_test VALUES (3, 3.3), (4, 4.4); -- { serverError 290 } INSERT INTO 02418_test VALUES (1, 2.1), (2, 3.2), (3, 3.3); SELECT count() FROM 02418_test; -CREATE TABLE 02418_test_another (key UInt64, value Float64) Engine=KeeperMap('/' || currentDatabase() || '/test2418', 1, 4) PRIMARY KEY(key); +CREATE TABLE 02418_test_another (key UInt64, value Float64) Engine=KeeperMap('/' || currentDatabase() || '/test2418', 4) PRIMARY KEY(key); INSERT INTO 02418_test VALUES (4, 4.4); -- { serverError 290 } INSERT INTO 02418_test_another VALUES (4, 4.4); diff --git a/tests/queries/0_stateless/02419_keeper_map_primary_key.reference b/tests/queries/0_stateless/02419_keeper_map_primary_key.reference new file mode 100644 index 00000000000..8394d9f34a7 --- /dev/null +++ b/tests/queries/0_stateless/02419_keeper_map_primary_key.reference @@ -0,0 +1,6 @@ +1.1 +2.2 +1.1 +2.2 +1.1 +2.2 diff --git a/tests/queries/0_stateless/02419_keeper_map_primary_key.sh b/tests/queries/0_stateless/02419_keeper_map_primary_key.sh new file mode 100755 index 00000000000..d5ec3be58da --- /dev/null +++ b/tests/queries/0_stateless/02419_keeper_map_primary_key.sh @@ -0,0 +1,23 @@ +#!/usr/bin/env bash +# Tags: no-ordinary-database, no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS 02419_test SYNC;" + +test_primary_key() +{ + $CLICKHOUSE_CLIENT -nm -q " + CREATE TABLE 02419_test (key UInt64, value Float64) Engine=KeeperMap('/' || currentDatabase() || '/test2418', 3) PRIMARY KEY($1); + INSERT INTO 02419_test VALUES (1, 1.1), (2, 2.2); + SELECT value FROM 02419_test WHERE key = 1; + SELECT value FROM 02419_test WHERE key IN (2, 3); + DROP TABLE 02419_test SYNC; + " +} + +test_primary_key "sipHash64(key + 42) * 12212121212121" +test_primary_key "reverse(concat(CAST(key, 'String'), 'some string'))" +test_primary_key "hex(toFloat32(key))" From b6f8e067080dbb7929aee9154e34416fc6f4d286 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 8 Sep 2022 14:57:34 +0200 Subject: [PATCH 40/43] Typo fix --- src/Storages/StorageKeeperMap.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index de7456d0df1..fbc4d34b2e7 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -598,7 +598,7 @@ std::optional StorageKeeperMap::isTableValid() const { LOG_ERROR( log, - "Table definition does not match to the one stored in the path {}. Stored defintion: {}", + "Table definition does not match to the one stored in the path {}. Stored definition: {}", root_path, stored_metadata_string); table_is_valid = false; From 43b6f395fb201d0834105d6daaee511ab052e4f5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Sep 2022 06:38:44 +0000 Subject: [PATCH 41/43] Mark the test as long --- tests/queries/0_stateless/02416_keeper_map.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02416_keeper_map.sql b/tests/queries/0_stateless/02416_keeper_map.sql index a87ad6d914b..c191b539de6 100644 --- a/tests/queries/0_stateless/02416_keeper_map.sql +++ b/tests/queries/0_stateless/02416_keeper_map.sql @@ -1,4 +1,4 @@ --- Tags: no-ordinary-database, no-fasttest +-- Tags: no-ordinary-database, no-fasttest, long DROP TABLE IF EXISTS 02416_test SYNC; From 2472b9c4047eb666755368c91d0ca62fd1002b3f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Sep 2022 12:24:26 +0000 Subject: [PATCH 42/43] Mark primary key test as long --- tests/queries/0_stateless/02419_keeper_map_primary_key.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02419_keeper_map_primary_key.sh b/tests/queries/0_stateless/02419_keeper_map_primary_key.sh index d5ec3be58da..c43c5bb6408 100755 --- a/tests/queries/0_stateless/02419_keeper_map_primary_key.sh +++ b/tests/queries/0_stateless/02419_keeper_map_primary_key.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-ordinary-database, no-fasttest +# Tags: no-ordinary-database, no-fasttest, long CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From deee0d639fe479f00cdda11830182350e8e340cf Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 12 Sep 2022 14:43:52 +0000 Subject: [PATCH 43/43] Address PR comments --- src/Storages/StorageKeeperMap.cpp | 78 ++++++++++++----------- src/Storages/StorageKeeperMap.h | 4 +- tests/integration/test_keeper_map/test.py | 30 ++++++--- 3 files changed, 66 insertions(+), 46 deletions(-) diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index fbc4d34b2e7..3ae7cf7a7e4 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -55,22 +55,11 @@ namespace ErrorCodes namespace { -std::string_view getBaseName(const std::string_view path) -{ - auto last_slash = path.find_last_of('/'); - if (last_slash == std::string_view::npos) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to get basename of path '{}'", path); - - return path.substr(last_slash + 1); -} - std::string formattedAST(const ASTPtr & ast) { if (!ast) return ""; - WriteBufferFromOwnString buf; - formatAST(*ast, buf, false, true); - return buf.str(); + return serializeAST(*ast); } void verifyTableId(const StorageID & table_id) @@ -126,7 +115,7 @@ public: ++idx; } - auto key = base64Encode(wb_key.str(), true); + auto key = base64Encode(wb_key.str(), /* url_encoding */ true); new_values[std::move(key)] = std::move(wb_value.str()); } } @@ -146,10 +135,9 @@ public: // (e.g if parallel insert queries are being run) if (keys_limit != 0) { - Coordination::Stat root_stat; - zookeeper->get(storage.rootKeeperPath(), &root_stat); - // exclude metadata node - current_keys_num = root_stat.numChildren - 1; + Coordination::Stat data_stat; + zookeeper->get(storage.dataPath(), &data_stat); + current_keys_num = data_stat.numChildren; } std::vector>> exist_responses; @@ -231,7 +219,7 @@ public: auto raw_keys = serializeKeysToRawString(it, end, key_column_type, max_block_size); for (auto & raw_key : raw_keys) - raw_key = base64Encode(raw_key, true); + raw_key = base64Encode(raw_key, /* url_encoding */ true); return storage.getBySerializedKeys(raw_keys, nullptr); } @@ -299,6 +287,8 @@ StorageKeeperMap::StorageKeeperMap( auto root_path_fs = fs::path(path_prefix) / std::string_view{root_path}.substr(1); root_path = root_path_fs.generic_string(); + data_path = root_path_fs / "data"; + auto metadata_path_fs = root_path_fs / "metadata"; metadata_path = metadata_path_fs; tables_path = metadata_path_fs / "tables"; @@ -381,6 +371,8 @@ StorageKeeperMap::StorageKeeperMap( if (code == Coordination::Error::ZOK) { + // metadata now should be guaranteed to exist because we added our UUID to the tables_path + client->createIfNotExists(data_path, ""); table_is_valid = true; return; } @@ -444,15 +436,7 @@ Pipe StorageKeeperMap::read( auto client = getClient(); if (all_scan) - { - auto children = std::make_shared>(client->getChildren(root_path)); - std::erase_if(*children, [&](const std::string_view key) - { - return fullPathForKey(key) == metadata_path; - }); - - return process_keys(std::move(children)); - } + return process_keys(std::make_shared>(client->getChildren(data_path))); return process_keys(std::move(filtered_keys)); } @@ -467,17 +451,18 @@ void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont { checkTable(); auto client = getClient(); - client->tryRemoveChildrenRecursive(root_path, true, zkutil::RemoveException{getBaseName(metadata_path), /*remove_subtree_*/ false}); + client->tryRemoveChildrenRecursive(data_path, true); } bool StorageKeeperMap::dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock) { - zookeeper->removeChildrenRecursive(root_path, zkutil::RemoveException{getBaseName(metadata_path), /*remove_subtree_*/ false}); + zookeeper->removeChildrenRecursive(data_path); bool completely_removed = false; Coordination::Requests ops; ops.emplace_back(zkutil::makeRemoveRequest(metadata_drop_lock->getPath(), -1)); ops.emplace_back(zkutil::makeRemoveRequest(dropped_path, -1)); + ops.emplace_back(zkutil::makeRemoveRequest(data_path, -1)); ops.emplace_back(zkutil::makeRemoveRequest(metadata_path, -1)); Coordination::Responses responses; @@ -489,7 +474,7 @@ bool StorageKeeperMap::dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::E { metadata_drop_lock->setAlreadyRemoved(); completely_removed = true; - LOG_INFO(log, "Metadata in {} was successfully removed from ZooKeeper", metadata_path); + LOG_INFO(log, "Metadata ({}) and data ({}) was successfully removed from ZooKeeper", metadata_path, data_path); break; } case ZNONODE: @@ -551,20 +536,20 @@ zkutil::ZooKeeperPtr StorageKeeperMap::getClient() const else zookeeper_client = getContext()->getAuxiliaryZooKeeper(zookeeper_name); - zookeeper_client->sync(rootKeeperPath()); + zookeeper_client->sync(root_path); } return zookeeper_client; } -const std::string & StorageKeeperMap::rootKeeperPath() const +const std::string & StorageKeeperMap::dataPath() const { - return root_path; + return data_path; } std::string StorageKeeperMap::fullPathForKey(const std::string_view key) const { - return fmt::format("{}/{}", root_path, key); + return fs::path(data_path) / key; } UInt64 StorageKeeperMap::keysLimit() const @@ -605,15 +590,35 @@ std::optional StorageKeeperMap::isTableValid() const return; } - // validate all metadata nodes are present + // validate all metadata and data nodes are present Coordination::Requests requests; requests.push_back(zkutil::makeCheckRequest(table_path, -1)); + requests.push_back(zkutil::makeCheckRequest(data_path, -1)); requests.push_back(zkutil::makeCheckRequest(dropped_path, -1)); Coordination::Responses responses; client->tryMulti(requests, responses); - table_is_valid = responses[0]->error == Coordination::Error::ZOK && responses[1]->error == Coordination::Error::ZNONODE; + table_is_valid = false; + if (responses[0]->error != Coordination::Error::ZOK) + { + LOG_ERROR(log, "Table node ({}) is missing", table_path); + return; + } + + if (responses[1]->error != Coordination::Error::ZOK) + { + LOG_ERROR(log, "Data node ({}) is missing", data_path); + return; + } + + if (responses[2]->error == Coordination::Error::ZOK) + { + LOG_ERROR(log, "Tables with root node {} are being dropped", root_path); + return; + } + + table_is_valid = true; } catch (const Coordination::Exception & e) { @@ -711,6 +716,7 @@ void StorageKeeperMap::checkTableCanBeRenamed(const StorageID & new_name) const void StorageKeeperMap::rename(const String & /*new_path_to_table_data*/, const StorageID & new_table_id) { checkTableCanBeRenamed(new_table_id); + renameInMemory(new_table_id); } namespace diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 504d3df087d..87861362e42 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -66,7 +66,7 @@ public: } zkutil::ZooKeeperPtr getClient() const; - const std::string & rootKeeperPath() const; + const std::string & dataPath() const; std::string fullPathForKey(std::string_view key) const; UInt64 keysLimit() const; @@ -110,6 +110,8 @@ private: std::string root_path; std::string primary_key; + std::string data_path; + std::string metadata_path; std::string tables_path; diff --git a/tests/integration/test_keeper_map/test.py b/tests/integration/test_keeper_map/test.py index 7801dd3c57f..8f515077e8f 100644 --- a/tests/integration/test_keeper_map/test.py +++ b/tests/integration/test_keeper_map/test.py @@ -52,29 +52,41 @@ def test_create_keeper_map(started_cluster): ) zk_client = get_genuine_zk() - def assert_children_size(expected_size): - assert len(zk_client.get_children("/test_keeper_map/test1")) == expected_size + def assert_children_size(path, expected_size): + assert len(zk_client.get_children(path)) == expected_size - assert_children_size(1) + def assert_root_children_size(expected_size): + assert_children_size("/test_keeper_map/test1", expected_size) + + def assert_data_children_size(expected_size): + assert_children_size("/test_keeper_map/test1/data", expected_size) + + assert_root_children_size(2) + assert_data_children_size(0) node.query("INSERT INTO test_keeper_map VALUES (1, 11)") - assert_children_size(2) + assert_data_children_size(1) node.query( "CREATE TABLE test_keeper_map_another (key UInt64, value UInt64) ENGINE = KeeperMap('/test1') PRIMARY KEY(key);" ) - assert_children_size(2) + assert_root_children_size(2) + assert_data_children_size(1) + node.query("INSERT INTO test_keeper_map_another VALUES (1, 11)") - assert_children_size(2) + assert_root_children_size(2) + assert_data_children_size(1) node.query("INSERT INTO test_keeper_map_another VALUES (2, 22)") - assert_children_size(3) + assert_root_children_size(2) + assert_data_children_size(2) node.query("DROP TABLE test_keeper_map SYNC") - assert_children_size(3) + assert_root_children_size(2) + assert_data_children_size(2) node.query("DROP TABLE test_keeper_map_another SYNC") - assert_children_size(0) + assert_root_children_size(0) zk_client.stop()