From aacd53516e7e94af89660e6ee5472a9427dfbc54 Mon Sep 17 00:00:00 2001 From: Bharat Nallan Chakravarthy Date: Wed, 9 Aug 2023 20:44:30 -0700 Subject: [PATCH 1/7] support truncate database --- src/Access/Common/AccessType.h | 2 +- src/Interpreters/InterpreterDropQuery.cpp | 21 ++++++++++++--------- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 06507fd85c8..c8ac0c4a210 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -95,7 +95,7 @@ enum class AccessType M(CREATE_NAMED_COLLECTION, "", NAMED_COLLECTION, NAMED_COLLECTION_ADMIN) /* allows to execute CREATE NAMED COLLECTION */ \ M(CREATE, "", GROUP, ALL) /* allows to execute {CREATE|ATTACH} */ \ \ - M(DROP_DATABASE, "", DATABASE, DROP) /* allows to execute {DROP|DETACH} DATABASE */\ + M(DROP_DATABASE, "", DATABASE, DROP) /* allows to execute {DROP|DETACH|TRUNCATE} DATABASE */\ M(DROP_TABLE, "", TABLE, DROP) /* allows to execute {DROP|DETACH} TABLE */\ M(DROP_VIEW, "", VIEW, DROP) /* allows to execute {DROP|DETACH} TABLE for views; implicitly enabled by the grant DROP_TABLE */\ diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index ed927d550a8..bc7982dad40 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -329,13 +329,11 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, database = tryGetDatabase(database_name, query.if_exists); if (database) { - if (query.kind == ASTDropQuery::Kind::Truncate) - { - throw Exception(ErrorCodes::SYNTAX_ERROR, "Unable to truncate database"); - } - else if (query.kind == ASTDropQuery::Kind::Detach || query.kind == ASTDropQuery::Kind::Drop) + if (query.kind == ASTDropQuery::Kind::Detach || query.kind == ASTDropQuery::Kind::Drop + || query.kind == ASTDropQuery::Kind::Truncate) { bool drop = query.kind == ASTDropQuery::Kind::Drop; + bool truncate = query.kind == ASTDropQuery::Kind::Truncate; getContext()->checkAccess(AccessType::DROP_DATABASE, database_name); if (query.kind == ASTDropQuery::Kind::Detach && query.permanently) @@ -348,6 +346,8 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, { ASTDropQuery query_for_table; query_for_table.kind = query.kind; + if (truncate) + query_for_table.kind = ASTDropQuery::Kind::Drop; query_for_table.if_exists = true; query_for_table.setDatabase(database_name); query_for_table.sync = query.sync; @@ -371,12 +371,13 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, query_for_table.is_dictionary = table.second; DatabasePtr db; UUID table_to_wait = UUIDHelpers::Nil; + //FAILS here executeToTableImpl(table_context, query_for_table, db, table_to_wait); uuids_to_wait.push_back(table_to_wait); } } - if (!drop && query.sync) + if ((!drop || !truncate) && query.sync) { /// Avoid "some tables are still in use" when sync mode is enabled for (const auto & table_uuid : uuids_to_wait) @@ -386,11 +387,13 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, /// Protects from concurrent CREATE TABLE queries auto db_guard = DatabaseCatalog::instance().getExclusiveDDLGuardForDatabase(database_name); - if (!drop) + if (!drop || !truncate) database->assertCanBeDetached(true); - /// DETACH or DROP database itself - DatabaseCatalog::instance().detachDatabase(getContext(), database_name, drop, database->shouldBeEmptyOnDetach()); + /// DETACH or DROP database itself. + /// If TRUNCATE skip dropping the database. + if(!truncate) + DatabaseCatalog::instance().detachDatabase(getContext(), database_name, drop, database->shouldBeEmptyOnDetach()); } } From 01902a79dd970c4c3b97944fdb76f83a3a90061b Mon Sep 17 00:00:00 2001 From: Bharat Nallan Chakravarthy Date: Wed, 9 Aug 2023 22:42:45 -0700 Subject: [PATCH 2/7] add initial tests --- .../02842_truncate_database.reference | 22 ++++++ .../0_stateless/02842_truncate_database.sql | 74 +++++++++++++++++++ 2 files changed, 96 insertions(+) create mode 100644 tests/queries/0_stateless/02842_truncate_database.reference create mode 100644 tests/queries/0_stateless/02842_truncate_database.sql diff --git a/tests/queries/0_stateless/02842_truncate_database.reference b/tests/queries/0_stateless/02842_truncate_database.reference new file mode 100644 index 00000000000..71f52bcd1da --- /dev/null +++ b/tests/queries/0_stateless/02842_truncate_database.reference @@ -0,0 +1,22 @@ +0 +0 +0 +0 +0 +dest_dictionary test_truncate_database 0 +1 First +=== TABLES IN test_truncate_database === +dest_dictionary +dest_view_log +dest_view_memory +dest_view_merge_tree +dest_view_stripe_log +dest_view_tiny_log +source_table_dictionary +source_table_log +source_table_memory +source_table_merge_tree +source_table_stripe_log +source_table_tiny_log +=== DICTIONARIES IN test_truncate_database === +dest_dictionary diff --git a/tests/queries/0_stateless/02842_truncate_database.sql b/tests/queries/0_stateless/02842_truncate_database.sql new file mode 100644 index 00000000000..d79289823b2 --- /dev/null +++ b/tests/queries/0_stateless/02842_truncate_database.sql @@ -0,0 +1,74 @@ +DROP DATABASE IF EXISTS test_truncate_database; + +-- test TRUNCATE DATABASE operation. +-- create tables, views and dictionary and populate them. Then try truncating the database. +-- all tables, views and dictionaries should be removed leaving an empty database +CREATE DATABASE test_truncate_database; +USE test_truncate_database; + +-- create tables with several different types of table engines +CREATE TABLE source_table_memory (x UInt16) ENGINE = Memory; +CREATE TABLE source_table_log (x UInt16) ENGINE = Log; +CREATE TABLE source_table_tiny_log (x UInt16) ENGINE = TinyLog; +CREATE TABLE source_table_stripe_log (x UInt16) ENGINE = StripeLog; +CREATE TABLE source_table_merge_tree (x UInt16) ENGINE = MergeTree ORDER BY x PARTITION BY x; +-- create dictionary source table +CREATE TABLE source_table_dictionary +( + id UInt64, + value String +) ENGINE = Memory(); + +-- insert data into the tables +INSERT INTO source_table_memory SELECT * FROM system.numbers LIMIT 10; +INSERT INTO source_table_log SELECT * FROM system.numbers LIMIT 10; +INSERT INTO source_table_tiny_log SELECT * FROM system.numbers LIMIT 10; +INSERT INTO source_table_stripe_log SELECT * FROM system.numbers LIMIT 10; +INSERT INTO source_table_merge_tree SELECT * FROM system.numbers LIMIT 10; +INSERT INTO source_table_dictionary VALUES (1, 'First'); + + +-- create view based on the tables +CREATE VIEW dest_view_memory (x UInt64) AS SELECT * FROM source_table_memory; +CREATE VIEW dest_view_log (x UInt64) AS SELECT * FROM source_table_log; +CREATE VIEW dest_view_tiny_log (x UInt64) AS SELECT * FROM source_table_tiny_log; +CREATE VIEW dest_view_stripe_log (x UInt64) AS SELECT * FROM source_table_stripe_log; +CREATE VIEW dest_view_merge_tree (x UInt64) AS SELECT * FROM source_table_merge_tree; +-- create dictionary based on source table +CREATE DICTIONARY dest_dictionary +( + id UInt64, + value String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() DB 'test_truncate_database' TABLE 'source_table_dictionary')) +LAYOUT(FLAT()) +LIFETIME(MIN 0 MAX 1000); + + +SELECT * FROM dest_view_memory ORDER BY x LIMIT 1; +SELECT * FROM dest_view_log ORDER BY x LIMIT 1; +SELECT * FROM dest_view_tiny_log ORDER BY x LIMIT 1; +SELECT * FROM dest_view_stripe_log ORDER BY x LIMIT 1; +SELECT * FROM dest_view_merge_tree ORDER BY x LIMIT 1; +SELECT name, database, element_count FROM system.dictionaries WHERE database = 'test_truncate_database' AND name = 'dest_dictionary'; +SELECT * FROM dest_dictionary; +SELECT '=== TABLES IN test_truncate_database ==='; +SHOW TABLES FROM test_truncate_database; +SELECT '=== DICTIONARIES IN test_truncate_database ==='; +SHOW DICTIONARIES FROM test_truncate_database; + +TRUNCATE DATABASE test_truncate_database; + +SELECT * FROM dest_view_set ORDER BY x LIMIT 1; -- {serverError 60} +SELECT * FROM dest_view_memory ORDER BY x LIMIT 1; -- {serverError 60} +SELECT * FROM dest_view_log ORDER BY x LIMIT 1; -- {serverError 60} +SELECT * FROM dest_view_tiny_log ORDER BY x LIMIT 1; -- {serverError 60} +SELECT * FROM dest_view_stripe_log ORDER BY x LIMIT 1; -- {serverError 60} +SELECT * FROM dest_view_merge_tree ORDER BY x LIMIT 1; -- {serverError 60} +SELECT name, database, element_count FROM system.dictionaries WHERE database = 'test_truncate_database' AND name = 'dest_dictionary'; +SELECT * FROM dest_dictionary; -- {serverError 60} +SHOW TABLES FROM test_truncate_database; +SHOW DICTIONARIES FROM test_truncate_database; + +DROP DATABASE test_truncate_database; From 214c9e444b14f21346adc7661a572b532d5bbb1d Mon Sep 17 00:00:00 2001 From: Bharat Nallan Chakravarthy Date: Wed, 9 Aug 2023 22:50:19 -0700 Subject: [PATCH 3/7] add some more comments --- src/Interpreters/InterpreterDropQuery.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index bc7982dad40..d9b1965d37c 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -334,6 +334,7 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, { bool drop = query.kind == ASTDropQuery::Kind::Drop; bool truncate = query.kind == ASTDropQuery::Kind::Truncate; + getContext()->checkAccess(AccessType::DROP_DATABASE, database_name); if (query.kind == ASTDropQuery::Kind::Detach && query.permanently) @@ -346,6 +347,7 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, { ASTDropQuery query_for_table; query_for_table.kind = query.kind; + // For truncate operation on database, drop the tables if (truncate) query_for_table.kind = ASTDropQuery::Kind::Drop; query_for_table.if_exists = true; @@ -371,12 +373,11 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, query_for_table.is_dictionary = table.second; DatabasePtr db; UUID table_to_wait = UUIDHelpers::Nil; - //FAILS here executeToTableImpl(table_context, query_for_table, db, table_to_wait); uuids_to_wait.push_back(table_to_wait); } } - + // only if operation is DETACH if ((!drop || !truncate) && query.sync) { /// Avoid "some tables are still in use" when sync mode is enabled @@ -386,12 +387,11 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, /// Protects from concurrent CREATE TABLE queries auto db_guard = DatabaseCatalog::instance().getExclusiveDDLGuardForDatabase(database_name); - + // only if operation is DETACH if (!drop || !truncate) database->assertCanBeDetached(true); - /// DETACH or DROP database itself. - /// If TRUNCATE skip dropping the database. + /// DETACH or DROP database itself. If TRUNCATE skip dropping/erasing the database. if(!truncate) DatabaseCatalog::instance().detachDatabase(getContext(), database_name, drop, database->shouldBeEmptyOnDetach()); } From 65bdc20a01c8c8a6dacfbb9123981b719b884d7c Mon Sep 17 00:00:00 2001 From: Bharat Nallan Chakravarthy Date: Wed, 9 Aug 2023 23:40:04 -0700 Subject: [PATCH 4/7] fix style check --- src/Interpreters/InterpreterDropQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index d9b1965d37c..f70ecebb341 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -392,7 +392,7 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, database->assertCanBeDetached(true); /// DETACH or DROP database itself. If TRUNCATE skip dropping/erasing the database. - if(!truncate) + if (!truncate) DatabaseCatalog::instance().detachDatabase(getContext(), database_name, drop, database->shouldBeEmptyOnDetach()); } } From 8cd9f3b912c250ed586292b4dc1ae126a15a546e Mon Sep 17 00:00:00 2001 From: Bharat Nallan Chakravarthy Date: Wed, 9 Aug 2023 23:57:17 -0700 Subject: [PATCH 5/7] add docs --- docs/en/sql-reference/statements/truncate.md | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/truncate.md b/docs/en/sql-reference/statements/truncate.md index 457031a2157..4b46210aa09 100644 --- a/docs/en/sql-reference/statements/truncate.md +++ b/docs/en/sql-reference/statements/truncate.md @@ -4,8 +4,9 @@ sidebar_position: 52 sidebar_label: TRUNCATE --- -# TRUNCATE Statement +# TRUNCATE Statements +## TRUNCATE TABLE ``` sql TRUNCATE TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] ``` @@ -21,3 +22,10 @@ You can specify how long (in seconds) to wait for inactive replicas to execute ` :::note If the `alter_sync` is set to `2` and some replicas are not active for more than the time, specified by the `replication_wait_for_inactive_replica_timeout` setting, then an exception `UNFINISHED` is thrown. ::: + +## TRUNCATE DATABASE +``` sql +TRUNCATE DATBASE [IF EXISTS] [db.]name [ON CLUSTER cluster] +``` + +Removes all tables from a database but keeps the database itself. When the clause `IF EXISTS` is omitted, the query returns an error if the database does not exist. From 73ea9b058f88a2d38cecd041ebfb2519e59ae5cd Mon Sep 17 00:00:00 2001 From: Bharat Nallan Chakravarthy Date: Thu, 10 Aug 2023 20:58:19 -0700 Subject: [PATCH 6/7] add integration tests --- .../test_truncate_database/__init__.py | 0 .../configs/distributed_servers.xml | 18 ++++++ .../configs/replicated_servers.xml | 21 +++++++ .../test_distributed.py | 53 ++++++++++++++++++ .../test_truncate_database/test_replicated.py | 56 +++++++++++++++++++ 5 files changed, 148 insertions(+) create mode 100644 tests/integration/test_truncate_database/__init__.py create mode 100644 tests/integration/test_truncate_database/configs/distributed_servers.xml create mode 100644 tests/integration/test_truncate_database/configs/replicated_servers.xml create mode 100644 tests/integration/test_truncate_database/test_distributed.py create mode 100644 tests/integration/test_truncate_database/test_replicated.py diff --git a/tests/integration/test_truncate_database/__init__.py b/tests/integration/test_truncate_database/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_truncate_database/configs/distributed_servers.xml b/tests/integration/test_truncate_database/configs/distributed_servers.xml new file mode 100644 index 00000000000..68b420f36b4 --- /dev/null +++ b/tests/integration/test_truncate_database/configs/distributed_servers.xml @@ -0,0 +1,18 @@ + + + + + + node1 + 9000 + + + + + node2 + 9000 + + + + + diff --git a/tests/integration/test_truncate_database/configs/replicated_servers.xml b/tests/integration/test_truncate_database/configs/replicated_servers.xml new file mode 100644 index 00000000000..8e318d385c3 --- /dev/null +++ b/tests/integration/test_truncate_database/configs/replicated_servers.xml @@ -0,0 +1,21 @@ + + + + + true + + node1 + 9000 + + + node2 + 9000 + + + node3 + 9000 + + + + + diff --git a/tests/integration/test_truncate_database/test_distributed.py b/tests/integration/test_truncate_database/test_distributed.py new file mode 100644 index 00000000000..fed3d16d190 --- /dev/null +++ b/tests/integration/test_truncate_database/test_distributed.py @@ -0,0 +1,53 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance( + "node1", main_configs=["configs/distributed_servers.xml"], with_zookeeper=True +) +node2 = cluster.add_instance( + "node2", with_zookeeper=True, main_configs=["configs/distributed_servers.xml"] +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + for node in (node1, node2): + node.query( + """ + CREATE DATABASE test; + CREATE TABLE test.local_table(id UInt32, val String) ENGINE = MergeTree ORDER BY id + """ + ) + + node1.query("INSERT INTO test.local_table VALUES (1, 'node1')") + node2.query("INSERT INTO test.local_table VALUES (2, 'node2')") + + node1.query( + "CREATE TABLE test.distributed(id UInt32, val String) ENGINE = Distributed(test_cluster, test, local_table)" + ) + node2.query( + "CREATE TABLE test.distributed(id UInt32, val String) ENGINE = Distributed(test_cluster, test, local_table)" + ) + + yield cluster + + finally: + cluster.shutdown() + + +def test_truncate_database_distributed(started_cluster): + query1 = "SELECT count() FROM test.distributed WHERE (id, val) IN ((1, 'node1'), (2, 'a'), (3, 'b'))" + query2 = "SELECT sum((id, val) IN ((1, 'node1'), (2, 'a'), (3, 'b'))) FROM test.distributed" + assert node1.query(query1) == "1\n" + assert node1.query(query2) == "1\n" + assert node2.query(query1) == "1\n" + assert node2.query(query2) == "1\n" + assert node2.query("SHOW DATABASES LIKE 'test'") == "test\n" + node1.query("TRUNCATE DATABASE test ON CLUSTER test_cluster SYNC") + assert node2.query("SHOW TABLES FROM test") == "" diff --git a/tests/integration/test_truncate_database/test_replicated.py b/tests/integration/test_truncate_database/test_replicated.py new file mode 100644 index 00000000000..59830d44378 --- /dev/null +++ b/tests/integration/test_truncate_database/test_replicated.py @@ -0,0 +1,56 @@ +import time + +import pytest +from helpers.cluster import ClickHouseCluster + + +def fill_nodes(nodes, shard): + for node in nodes: + node.query( + """ + CREATE DATABASE test; + + CREATE TABLE test.test_table(date Date, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{shard}/replicated/test_table', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date); + """.format( + shard=shard, replica=node.name + ) + ) + + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", with_zookeeper=True, main_configs=["configs/replicated_servers.xml"] +) +node2 = cluster.add_instance( + "node2", with_zookeeper=True, main_configs=["configs/replicated_servers.xml"] +) +node3 = cluster.add_instance( + "node3", with_zookeeper=True, main_configs=["configs/replicated_servers.xml"] +) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + + fill_nodes([node1, node2, node3], 1) + + yield cluster + + except Exception as ex: + print(ex) + + finally: + cluster.shutdown() + + +def test_truncate_database_replicated(start_cluster): + node1.query( + "INSERT INTO test.test_table SELECT number, toString(number) FROM numbers(100)" + ) + assert node2.query("SELECT id FROM test.test_table LIMIT 1") == "0\n" + assert node3.query("SHOW DATABASES LIKE 'test'") == "test\n" + node3.query("TRUNCATE DATABASE test ON CLUSTER test_cluster SYNC") + assert node2.query("SHOW TABLES FROM test") == "" From 97702f2ea6ba898f5200e102134c22d1903ceb42 Mon Sep 17 00:00:00 2001 From: Bharat Nallan Chakravarthy Date: Fri, 11 Aug 2023 11:42:59 -0700 Subject: [PATCH 7/7] fix test --- tests/queries/0_stateless/02842_truncate_database.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02842_truncate_database.sql b/tests/queries/0_stateless/02842_truncate_database.sql index d79289823b2..a767acba14c 100644 --- a/tests/queries/0_stateless/02842_truncate_database.sql +++ b/tests/queries/0_stateless/02842_truncate_database.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel + DROP DATABASE IF EXISTS test_truncate_database; -- test TRUNCATE DATABASE operation.