Add support for drop/truncate and tests

This commit is contained in:
Antonio Andelic 2022-08-10 07:24:56 +00:00
parent 617ef00908
commit 4023d4a37a
7 changed files with 114 additions and 43 deletions

View File

@ -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; Strings children;
if (tryGetChildren(path, children) != Coordination::Error::ZOK) 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(); 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())) 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); batch.push_back(child_path);
ops.emplace_back(zkutil::makeRemoveRequest(child_path, -1)); ops.emplace_back(zkutil::makeRemoveRequest(child_path, -1));
} }

View File

@ -238,7 +238,7 @@ public:
/// If probably_flat is true, this method will optimistically try to remove children non-recursive /// 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. /// and will fall back to recursive removal if it gets ZNOTEMPTY for some child.
/// Returns true if no kind of fallback happened. /// 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). /// Remove all children nodes (non recursive).
void removeChildren(const std::string & path); void removeChildren(const std::string & path);

View File

@ -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) for (size_t i = 0; i < header.columns(); ++i)
{ {
const auto & serialization = header.getByPosition(i).type->getDefaultSerialization(); 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); serialization->deserializeBinary(*columns[i], i == key_pos ? key_buffer : value_buffer);
} }
} }

View File

@ -82,10 +82,11 @@ std::string_view getBaseName(const std::string_view path)
struct ZooKeeperLock 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_path(std::move(lock_path_)), client(std::move(client_))
{ {
lock(); if (!defer_lock)
lock();
} }
~ZooKeeperLock() ~ZooKeeperLock()
@ -120,7 +121,7 @@ struct ZooKeeperLock
void unlock() void unlock()
{ {
assert(locked); assert(locked);
client->remove(sequence_path); client->tryRemove(sequence_path);
} }
private: private:
@ -307,11 +308,13 @@ StorageKeeperMap::StorageKeeperMap(
ContextPtr context, ContextPtr context,
const StorageID & table_id, const StorageID & table_id,
const StorageInMemoryMetadata & metadata, const StorageInMemoryMetadata & metadata,
bool attach,
std::string_view primary_key_, std::string_view primary_key_,
std::string_view keeper_path_, std::string_view keeper_path_,
const std::string & hosts, const std::string & hosts,
bool create_missing_root_path, 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)) : IKeyValueStorage(table_id), keeper_path(keeper_path_), primary_key(primary_key_), zookeeper_client(getZooKeeperClient(hosts, context))
{ {
setInMemoryMetadata(metadata); setInMemoryMetadata(metadata);
@ -323,6 +326,12 @@ StorageKeeperMap::StorageKeeperMap(
auto client = getClient(); auto client = getClient();
if (attach)
{
// validate all metadata nodes are present
return;
}
if (keeper_path != "/" && !client->exists(keeper_path)) if (keeper_path != "/" && !client->exists(keeper_path))
{ {
if (!create_missing_root_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 // create metadata nodes
std::filesystem::path root_path{keeper_path}; std::filesystem::path root_path{keeper_path};
auto metadata_path_fs = root_path / "__ch_metadata"; auto metadata_path_fs = root_path / "__ch_metadata";
metadata_path = metadata_path_fs; metadata_path = metadata_path_fs;
client->createIfNotExists(metadata_path, ""); client->create(metadata_path, "", zkutil::CreateMode::Persistent);
lock_path = metadata_path_fs / "lock"; 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 keys_limit_path = metadata_path_fs / "keys_limit";
auto status = client->tryCreate(keys_limit_path, toString(keys_limit_), zkutil::CreateMode::Persistent); client->create(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<UInt64>(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);
}
} }
@ -445,6 +446,20 @@ SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const Storage
return std::make_shared<StorageKeeperMapSink>(*this, metadata_snapshot); return std::make_shared<StorageKeeperMapSink>(*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 zkutil::ZooKeeperPtr & StorageKeeperMap::getClient() const
{ {
if (zookeeper_client->expired()) if (zookeeper_client->expired())
@ -563,11 +578,12 @@ StoragePtr create(const StorageFactory::Arguments & args)
if (engine_args.empty() || engine_args.size() > 4) if (engine_args.empty() || engine_args.size() > 4)
throw Exception( throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, 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" "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" "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); default_host);
auto keeper_path = checkAndGetLiteralArgument<std::string>(engine_args[0], "keeper_path"); auto keeper_path = checkAndGetLiteralArgument<std::string>(engine_args[0], "keeper_path");
@ -584,6 +600,10 @@ StoragePtr create(const StorageFactory::Arguments & args)
if (engine_args.size() > 3) if (engine_args.size() > 3)
create_missing_root_path = checkAndGetLiteralArgument<UInt64>(engine_args[3], "create_missing_root_path"); create_missing_root_path = checkAndGetLiteralArgument<UInt64>(engine_args[3], "create_missing_root_path");
bool remove_existing_data = false;
if (engine_args.size() > 4)
create_missing_root_path = checkAndGetLiteralArgument<UInt64>(engine_args[4], "create_missing_root_path");
StorageInMemoryMetadata metadata; StorageInMemoryMetadata metadata;
metadata.setColumns(args.columns); metadata.setColumns(args.columns);
metadata.setConstraints(args.constraints); 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); throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS);
return std::make_shared<StorageKeeperMap>( return std::make_shared<StorageKeeperMap>(
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);
} }
} }

View File

@ -17,16 +17,17 @@ namespace DB
class StorageKeeperMap final : public IKeyValueStorage class StorageKeeperMap final : public IKeyValueStorage
{ {
public: public:
// TODO(antonio2368): add setting to control creating if keeper_path doesn't exist
StorageKeeperMap( StorageKeeperMap(
ContextPtr context, ContextPtr context,
const StorageID & table_id, const StorageID & table_id,
const StorageInMemoryMetadata & metadata, const StorageInMemoryMetadata & metadata,
bool attach,
std::string_view primary_key_, std::string_view primary_key_,
std::string_view keeper_path_, std::string_view keeper_path_,
const std::string & hosts, const std::string & hosts,
bool create_missing_root_path, bool create_missing_root_path,
size_t keys_limit); size_t keys_limit,
bool remove_existing_data);
Pipe read( Pipe read(
const Names & column_names, const Names & column_names,
@ -39,6 +40,9 @@ public:
SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; 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"; } std::string getName() const override { return "KeeperMap"; }
Names getPrimaryKey() const override { return {primary_key}; } Names getPrimaryKey() const override { return {primary_key}; }

View File

@ -0,0 +1,6 @@
1
1
1
1
1 1 1 1 1
1

View File

@ -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;