Merge pull request #52687 from alkorgun/fix-named-collections-on-cluster-23.7

Fix named collections on cluster 23.7
This commit is contained in:
Anton Popov 2023-08-03 14:50:28 +02:00 committed by GitHub
commit 92cb5e9ae3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
29 changed files with 366 additions and 59 deletions

View File

@ -200,8 +200,8 @@ Templates:
- [Server Setting](_description_templates/template-server-setting.md)
- [Database or Table engine](_description_templates/template-engine.md)
- [System table](_description_templates/template-system-table.md)
- [Data type](_description_templates/data-type.md)
- [Statement](_description_templates/statement.md)
- [Data type](_description_templates/template-data-type.md)
- [Statement](_description_templates/template-statement.md)
<a name="how-to-build-docs"/>

View File

@ -141,6 +141,10 @@ Runs [stateful functional tests](tests.md#functional-tests). Treat them in the s
Runs [integration tests](tests.md#integration-tests).
## Bugfix validate check
Checks that either a new test (functional or integration) or there some changed tests that fail with the binary built on master branch. This check is triggered when pull request has "pr-bugfix" label.
## Stress Test
Runs stateless functional tests concurrently from several clients to detect
concurrency-related errors. If it fails:

View File

@ -22,7 +22,7 @@ CREATE TABLE deltalake
- `url` — Bucket url with path to the existing Delta Lake table.
- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file.
Engine parameters can be specified using [Named Collections](../../../operations/named-collections.md)
Engine parameters can be specified using [Named Collections](/docs/en/operations/named-collections.md).
**Example**

View File

@ -22,7 +22,7 @@ CREATE TABLE hudi_table
- `url` — Bucket url with the path to an existing Hudi table.
- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file.
Engine parameters can be specified using [Named Collections](../../../operations/named-collections.md)
Engine parameters can be specified using [Named Collections](/docs/en/operations/named-collections.md).
**Example**

View File

@ -36,6 +36,8 @@ These `ALTER` statements modify entities related to role-based access control:
[ALTER TABLE ... MODIFY COMMENT](/docs/en/sql-reference/statements/alter/comment.md) statement adds, modifies, or removes comments to the table, regardless if it was set before or not.
[ALTER NAMED COLLECTION](/docs/en/sql-reference/statements/alter/named-collection.md) statement modifies [Named Collections](/docs/en/operations/named-collections.md).
## Mutations
`ALTER` queries that are intended to manipulate table data are implemented with a mechanism called “mutations”, most notably [ALTER TABLE … DELETE](/docs/en/sql-reference/statements/alter/delete.md) and [ALTER TABLE … UPDATE](/docs/en/sql-reference/statements/alter/update.md). They are asynchronous background processes similar to merges in [MergeTree](/docs/en/engines/table-engines/mergetree-family/index.md) tables that to produce new “mutated” versions of parts.

View File

@ -0,0 +1,30 @@
---
slug: /en/sql-reference/statements/alter/named-collection
sidebar_label: NAMED COLLECTION
---
# ALTER NAMED COLLECTION
This query intends to modify already existing named collections.
**Syntax**
```sql
ALTER NAMED COLLECTION [IF EXISTS] name [ON CLUSTER cluster]
[ SET
key_name1 = 'some value',
key_name2 = 'some value',
key_name3 = 'some value',
... ] |
[ DELETE key_name4, key_name5, ... ]
```
**Example**
```sql
CREATE NAMED COLLECTION foobar AS a = '1', b = '2';
ALTER NAMED COLLECTION foobar SET a = '2', c = '3';
ALTER NAMED COLLECTION foobar DELETE b;
```

View File

@ -8,13 +8,14 @@ sidebar_label: CREATE
Create queries make a new entity of one of the following kinds:
- [DATABASE](../../../sql-reference/statements/create/database.md)
- [TABLE](../../../sql-reference/statements/create/table.md)
- [VIEW](../../../sql-reference/statements/create/view.md)
- [DICTIONARY](../../../sql-reference/statements/create/dictionary.md)
- [FUNCTION](../../../sql-reference/statements/create/function.md)
- [USER](../../../sql-reference/statements/create/user.md)
- [ROLE](../../../sql-reference/statements/create/role.md)
- [ROW POLICY](../../../sql-reference/statements/create/row-policy.md)
- [QUOTA](../../../sql-reference/statements/create/quota.md)
- [SETTINGS PROFILE](../../../sql-reference/statements/create/settings-profile.md)
- [DATABASE](/docs/en/sql-reference/statements/create/database.md)
- [TABLE](/docs/en/sql-reference/statements/create/table.md)
- [VIEW](/docs/en/sql-reference/statements/create/view.md)
- [DICTIONARY](/docs/en/sql-reference/statements/create/dictionary.md)
- [FUNCTION](/docs/en/sql-reference/statements/create/function.md)
- [USER](/docs/en/sql-reference/statements/create/user.md)
- [ROLE](/docs/en/sql-reference/statements/create/role.md)
- [ROW POLICY](/docs/en/sql-reference/statements/create/row-policy.md)
- [QUOTA](/docs/en/sql-reference/statements/create/quota.md)
- [SETTINGS PROFILE](/docs/en/sql-reference/statements/create/settings-profile.md)
- [NAMED COLLECTION](/docs/en/sql-reference/statements/create/named-collection.md)

View File

@ -0,0 +1,34 @@
---
slug: /en/sql-reference/statements/create/named-collection
sidebar_label: NAMED COLLECTION
---
# CREATE NAMED COLLECTION
Creates a new named collection.
**Syntax**
```sql
CREATE NAMED COLLECTION [IF NOT EXISTS] name [ON CLUSTER cluster] AS
key_name1 = 'some value',
key_name2 = 'some value',
key_name3 = 'some value',
...
```
**Example**
```sql
CREATE NAMED COLLECTION foobar AS a = '1', b = '2';
```
**Related statements**
- [CREATE NAMED COLLECTION](https://clickhouse.com/docs/en/sql-reference/statements/alter/named-collection)
- [DROP NAMED COLLECTION](https://clickhouse.com/docs/en/sql-reference/statements/drop#drop-function)
**See Also**
- [Named collections guide](/docs/en/operations/named-collections.md)

View File

@ -119,3 +119,20 @@ DROP FUNCTION [IF EXISTS] function_name [on CLUSTER cluster]
CREATE FUNCTION linear_equation AS (x, k, b) -> k*x + b;
DROP FUNCTION linear_equation;
```
## DROP NAMED COLLECTION
Deletes a named collection.
**Syntax**
``` sql
DROP NAMED COLLECTION [IF EXISTS] name [on CLUSTER cluster]
```
**Example**
``` sql
CREATE NAMED COLLECTION foobar AS a = '1', b = '2';
DROP NAMED COLLECTION foobar;
```

View File

@ -21,7 +21,7 @@ iceberg(url [,aws_access_key_id, aws_secret_access_key] [,format] [,structure])
- `format` — The [format](/docs/en/interfaces/formats.md/#formats) of the file. By default `Parquet` is used.
- `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`.
Engine parameters can be specified using [Named Collections](../../operations/named-collections.md)
Engine parameters can be specified using [Named Collections](/docs/en/operations/named-collections.md).
**Returned value**

View File

@ -8,6 +8,7 @@
#include <Parsers/formatAST.h>
#include <Parsers/ASTCreateNamedCollectionQuery.h>
#include <Parsers/ASTAlterNamedCollectionQuery.h>
#include <Parsers/ASTDropNamedCollectionQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/parseQuery.h>
@ -225,24 +226,15 @@ public:
void remove(const std::string & collection_name)
{
if (!removeIfExists(collection_name))
auto collection_path = getMetadataPath(collection_name);
if (!fs::exists(collection_path))
{
throw Exception(
ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST,
"Cannot remove collection `{}`, because it doesn't exist",
collection_name);
}
}
bool removeIfExists(const std::string & collection_name)
{
auto collection_path = getMetadataPath(collection_name);
if (fs::exists(collection_path))
{
fs::remove(collection_path);
return true;
}
return false;
fs::remove(collection_path);
}
private:
@ -393,36 +385,64 @@ void loadIfNot()
return loadIfNotUnlocked(lock);
}
void removeFromSQL(const std::string & collection_name, ContextPtr context)
void removeFromSQL(const ASTDropNamedCollectionQuery & query, ContextPtr context)
{
auto lock = lockNamedCollectionsTransaction();
loadIfNotUnlocked(lock);
LoadFromSQL(context).remove(collection_name);
NamedCollectionFactory::instance().remove(collection_name);
}
void removeIfExistsFromSQL(const std::string & collection_name, ContextPtr context)
{
auto lock = lockNamedCollectionsTransaction();
loadIfNotUnlocked(lock);
LoadFromSQL(context).removeIfExists(collection_name);
NamedCollectionFactory::instance().removeIfExists(collection_name);
auto & instance = NamedCollectionFactory::instance();
if (!instance.exists(query.collection_name))
{
if (!query.if_exists)
{
throw Exception(
ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST,
"Cannot remove collection `{}`, because it doesn't exist",
query.collection_name);
}
return;
}
LoadFromSQL(context).remove(query.collection_name);
instance.remove(query.collection_name);
}
void createFromSQL(const ASTCreateNamedCollectionQuery & query, ContextPtr context)
{
auto lock = lockNamedCollectionsTransaction();
loadIfNotUnlocked(lock);
NamedCollectionFactory::instance().add(query.collection_name, LoadFromSQL(context).create(query));
auto & instance = NamedCollectionFactory::instance();
if (instance.exists(query.collection_name))
{
if (!query.if_not_exists)
{
throw Exception(
ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS,
"A named collection `{}` already exists",
query.collection_name);
}
return;
}
instance.add(query.collection_name, LoadFromSQL(context).create(query));
}
void updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr context)
{
auto lock = lockNamedCollectionsTransaction();
loadIfNotUnlocked(lock);
auto & instance = NamedCollectionFactory::instance();
if (!instance.exists(query.collection_name))
{
if (!query.if_exists)
{
throw Exception(
ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST,
"Cannot remove collection `{}`, because it doesn't exist",
query.collection_name);
}
return;
}
LoadFromSQL(context).update(query);
auto collection = NamedCollectionFactory::instance().getMutable(query.collection_name);
auto collection = instance.getMutable(query.collection_name);
auto collection_lock = collection->lock();
for (const auto & [name, value] : query.changes)

View File

@ -8,6 +8,7 @@ namespace DB
class ASTCreateNamedCollectionQuery;
class ASTAlterNamedCollectionQuery;
class ASTDropNamedCollectionQuery;
namespace NamedCollectionUtils
{
@ -26,8 +27,7 @@ void reloadFromConfig(const Poco::Util::AbstractConfiguration & config);
void loadFromSQL(ContextPtr context);
/// Remove collection as well as its metadata from `context->getPath() / named_collections /`.
void removeFromSQL(const std::string & collection_name, ContextPtr context);
void removeIfExistsFromSQL(const std::string & collection_name, ContextPtr context);
void removeFromSQL(const ASTDropNamedCollectionQuery & query, ContextPtr context);
/// Create a new collection from AST and put it to `context->getPath() / named_collections /`.
void createFromSQL(const ASTCreateNamedCollectionQuery & query, ContextPtr context);

View File

@ -218,7 +218,7 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const SnapshotFileInfo & snapsh
}
catch (...)
{
LOG_INFO(log, "Failed to delete lock file for {} from S3", snapshot_path);
LOG_INFO(log, "Failed to delete lock file for {} from S3", snapshot_file_info.path);
tryLogCurrentException(__PRETTY_FUNCTION__);
}
});

View File

@ -1,5 +1,4 @@
#include <Interpreters/InterpreterCreateNamedCollectionQuery.h>
#include <Parsers/ASTCreateNamedCollectionQuery.h>
#include <Access/ContextAccess.h>
#include <Interpreters/Context.h>

View File

@ -22,11 +22,7 @@ BlockIO InterpreterDropNamedCollectionQuery::execute()
return executeDDLQueryOnCluster(query_ptr, current_context, params);
}
if (query.if_exists)
NamedCollectionUtils::removeIfExistsFromSQL(query.collection_name, current_context);
else
NamedCollectionUtils::removeFromSQL(query.collection_name, current_context);
NamedCollectionUtils::removeFromSQL(query, current_context);
return {};
}

View File

@ -15,6 +15,8 @@ ASTPtr ASTAlterNamedCollectionQuery::clone() const
void ASTAlterNamedCollectionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "Alter NAMED COLLECTION ";
if (if_exists)
settings.ostr << "IF EXISTS ";
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(collection_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(settings);
if (!changes.empty())

View File

@ -18,6 +18,8 @@ ASTPtr ASTCreateNamedCollectionQuery::clone() const
void ASTCreateNamedCollectionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE NAMED COLLECTION ";
if (if_not_exists)
settings.ostr << "IF NOT EXISTS ";
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(collection_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(settings);

View File

@ -13,6 +13,7 @@ class ASTCreateNamedCollectionQuery : public IAST, public ASTQueryWithOnCluster
public:
std::string collection_name;
SettingsChanges changes;
bool if_not_exists = false;
String getID(char) const override { return "CreateNamedCollectionQuery"; }

View File

@ -13,6 +13,8 @@ ASTPtr ASTDropNamedCollectionQuery::clone() const
void ASTDropNamedCollectionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP NAMED COLLECTION ";
if (if_exists)
settings.ostr << "IF EXISTS ";
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(collection_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(settings);
}

View File

@ -13,8 +13,9 @@ bool ParserAlterNamedCollectionQuery::parseImpl(IParser::Pos & pos, ASTPtr & nod
{
ParserKeyword s_alter("ALTER");
ParserKeyword s_collection("NAMED COLLECTION");
ParserKeyword s_if_exists("IF EXISTS");
ParserKeyword s_on("ON");
ParserKeyword s_delete("DELETE");
ParserIdentifier name_p;
ParserSetQuery set_p;
ParserToken s_comma(TokenType::Comma);
@ -32,10 +33,13 @@ bool ParserAlterNamedCollectionQuery::parseImpl(IParser::Pos & pos, ASTPtr & nod
if (!s_collection.ignore(pos, expected))
return false;
if (s_if_exists.ignore(pos, expected))
if_exists = true;
if (!name_p.parse(pos, collection_name, expected))
return false;
if (ParserKeyword{"ON"}.ignore(pos, expected))
if (s_on.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;

View File

@ -1421,15 +1421,17 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
bool ParserCreateNamedCollectionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_create("CREATE");
ParserKeyword s_attach("ATTACH");
ParserKeyword s_named_collection("NAMED COLLECTION");
ParserKeyword s_if_not_exists("IF NOT EXISTS");
ParserKeyword s_on("ON");
ParserKeyword s_as("AS");
ParserToken s_comma(TokenType::Comma);
ParserIdentifier name_p;
ParserToken s_comma(TokenType::Comma);
String cluster_str;
bool if_not_exists = false;
ASTPtr collection_name;
String cluster_str;
if (!s_create.ignore(pos, expected))
return false;
@ -1437,10 +1439,13 @@ bool ParserCreateNamedCollectionQuery::parseImpl(Pos & pos, ASTPtr & node, Expec
if (!s_named_collection.ignore(pos, expected))
return false;
if (s_if_not_exists.ignore(pos, expected))
if_not_exists = true;
if (!name_p.parse(pos, collection_name, expected))
return false;
if (ParserKeyword{"ON"}.ignore(pos, expected))
if (s_on.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
@ -1465,7 +1470,9 @@ bool ParserCreateNamedCollectionQuery::parseImpl(Pos & pos, ASTPtr & node, Expec
auto query = std::make_shared<ASTCreateNamedCollectionQuery>();
tryGetIdentifierNameInto(collection_name, query->collection_name);
query->if_not_exists = if_not_exists;
query->changes = changes;
query->cluster = std::move(cluster_str);
node = query;
return true;

View File

@ -548,6 +548,7 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/// CREATE NAMED COLLECTION name [ON CLUSTER cluster]
class ParserCreateNamedCollectionQuery : public IParserBase
{
protected:

View File

@ -12,6 +12,7 @@ bool ParserDropNamedCollectionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node
ParserKeyword s_drop("DROP");
ParserKeyword s_collection("NAMED COLLECTION");
ParserKeyword s_if_exists("IF EXISTS");
ParserKeyword s_on("ON");
ParserIdentifier name_p;
String cluster_str;
@ -31,7 +32,7 @@ bool ParserDropNamedCollectionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node
if (!name_p.parse(pos, collection_name, expected))
return false;
if (ParserKeyword{"ON"}.ignore(pos, expected))
if (s_on.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;

View File

@ -114,9 +114,9 @@ namespace
{
if (next_slash_after_glob_pos == std::string::npos)
{
result.emplace_back(
result.emplace_back(StorageHDFS::PathWithInfo{
String(ls.file_info[i].mName),
StorageHDFS::PathInfo{ls.file_info[i].mLastMod, static_cast<size_t>(ls.file_info[i].mSize)});
StorageHDFS::PathInfo{ls.file_info[i].mLastMod, static_cast<size_t>(ls.file_info[i].mSize)}});
}
else
{

View File

@ -0,0 +1,22 @@
<clickhouse>
<remote_servers>
<cluster>
<shard>
<internal_replication>true</internal_replication>
<replica>
<host>clickhouse1</host>
<port>9000</port>
</replica>
<replica>
<host>clickhouse2</host>
<port>9000</port>
</replica>
<replica>
<host>clickhouse3</host>
<port>9000</port>
</replica>
</shard>
<allow_distributed_ddl_queries>true</allow_distributed_ddl_queries>
</cluster>
</remote_servers>
</clickhouse>

View File

@ -0,0 +1,12 @@
<clickhouse>
<users>
<default>
<password></password>
<profile>default</profile>
<quota>default</quota>
<named_collection_control>1</named_collection_control>
<show_named_collections>1</show_named_collections>
<show_named_collections_secrets>1</show_named_collections_secrets>
</default>
</users>
</clickhouse>

View File

@ -0,0 +1,148 @@
"""
Test cases:
--- execute on the first node
create named collection foobar as a=1, b=2;
create named collection if not exists foobar on cluster '{cluster}' as a=1, b=2, c=3;
create named collection collection_present_on_first_node as a=1, b=2, s='string', x=0, y=-1;
--- execute on any other node
alter named collection foobar on cluster '{cluster}' set a=2, c=3;
alter named collection foobar on cluster '{cluster}' delete b;
alter named collection foobar on cluster '{cluster}' set a=3 delete c;
alter named collection if exists collection_absent_ewerywhere on cluster '{cluster}' delete b;
alter named collection if exists collection_present_on_first_node on cluster '{cluster}' delete b;
--- execute on every node
select * from system.named_collections;
--- execute on any node
drop named collection foobar on cluster '{cluster}';
drop named collection if exists collection_absent_ewerywhere on cluster '{cluster}';
drop named collection if exists collection_present_on_first_node on cluster '{cluster}';
--- execute on every node
select * from system.named_collections;
"""
import logging
from json import dumps, loads
from functools import partial
import pytest
from helpers.cluster import ClickHouseCluster
dumps = partial(dumps, ensure_ascii=False)
NODE01, NODE02, NODE03 = "clickhouse1", "clickhouse2", "clickhouse3"
CHECK_STRING_VALUE = "Some ~`$tr!ng-_+=123@#%^&&()|?[]{}<🤡>.,\t\n:;"
STMT_CREATE = "CREATE NAMED COLLECTION"
STMT_ALTER = "ALTER NAMED COLLECTION"
STMT_DROP = "DROP NAMED COLLECTION"
SYSTEM_TABLE = "system.named_collections"
@pytest.fixture(scope="module")
def cluster():
try:
cluster = ClickHouseCluster(__file__)
common_kwargs = dict(
main_configs=[
"configs/config.d/cluster.xml",
],
user_configs=[
"configs/users.d/default.xml",
],
with_zookeeper=True,
stay_alive=True,
)
for name in [NODE01, NODE02, NODE03]:
cluster.add_instance(name, **common_kwargs)
logging.info("Starting cluster...")
cluster.start()
logging.info("Cluster started")
yield cluster
finally:
cluster.shutdown()
def test_create_alter_drop_on_cluster(cluster):
"""
Executes the set of queries and checks the final named collections state.
"""
q_count_collections = f"select count() from {SYSTEM_TABLE}"
def check_collections_empty():
for name, node in list(cluster.instances.items()):
assert (
"0" == node.query(q_count_collections).strip()
), f"{SYSTEM_TABLE} is not empty on {name}"
foobar_final_state = {"name": "foobar", "collection": {"a": "3"}}
collection_present_on_first_node_final_state = {
"name": "collection_present_on_first_node",
"collection": {"a": "1", "s": CHECK_STRING_VALUE, "x": "0", "y": "-1"},
}
expected_state = {
NODE01: [foobar_final_state, collection_present_on_first_node_final_state],
NODE02: [foobar_final_state],
NODE03: [foobar_final_state],
}
q_get_collections = f"select * from {SYSTEM_TABLE} order by name desc format JSON"
def check_state():
for name, node in list(cluster.instances.items()):
result = loads(node.query(q_get_collections))["data"]
logging.debug("%s ?= %s", dumps(result), dumps(expected_state[name]))
assert (
expected_state[name] == result
), f"invalid {SYSTEM_TABLE} content on {name}: {result}"
check_collections_empty()
# create executed on the first node
node = cluster.instances[NODE01]
node.query(f"{STMT_CREATE} foobar AS a=1, b=2")
node.query(
f"{STMT_CREATE} IF NOT EXISTS foobar ON CLUSTER 'cluster' AS a=1, b=2, c=3"
)
node.query(
f"{STMT_CREATE} collection_present_on_first_node AS a=1, b=2, s='{CHECK_STRING_VALUE}', x=0, y=-1"
)
# alter executed on the second node
node = cluster.instances[NODE02]
node.query(f"{STMT_ALTER} foobar ON CLUSTER 'cluster' SET a=2, c=3")
node.query(f"{STMT_ALTER} foobar ON CLUSTER 'cluster' DELETE b")
node.query(f"{STMT_ALTER} foobar ON CLUSTER 'cluster' SET a=3 DELETE c")
node.query(
f"{STMT_ALTER} IF EXISTS collection_absent_ewerywhere ON CLUSTER 'cluster' DELETE b"
)
node.query(
f"{STMT_ALTER} IF EXISTS collection_present_on_first_node ON CLUSTER 'cluster' DELETE b"
)
check_state()
for node in list(cluster.instances.values()):
node.restart_clickhouse()
check_state()
# drop executed on the third node
node = cluster.instances[NODE03]
node.query(f"{STMT_DROP} foobar ON CLUSTER 'cluster'")
node.query(
f"{STMT_DROP} IF EXISTS collection_absent_ewerywhere ON CLUSTER 'cluster'"
)
node.query(
f"{STMT_DROP} IF EXISTS collection_present_on_first_node ON CLUSTER 'cluster'"
)
check_collections_empty()
for node in list(cluster.instances.values()):
node.restart_clickhouse()
check_collections_empty()

View File

@ -99,6 +99,7 @@ BlockWriteOps
BlockWriteTime
Bool
BrokenDistributedFilesToInsert
Bugfix
BuildID
BuilderBinAarch
BuilderBinAmd
@ -1160,6 +1161,7 @@ brotli
bson
bsoneachrow
buffersize
bugfix
buildId
buildable
builtins