Merge pull request #53261 from bharatnc/ncb/truncate-db

support TRUNCATE DATABASE
This commit is contained in:
Alexey Milovidov 2023-08-18 08:22:32 +03:00 committed by GitHub
commit e7d0edfce6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 270 additions and 13 deletions

View File

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

View File

@ -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 */\

View File

@ -329,13 +329,12 @@ 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 +347,9 @@ 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;
query_for_table.setDatabase(database_name);
query_for_table.sync = query.sync;
@ -375,8 +377,8 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query,
uuids_to_wait.push_back(table_to_wait);
}
}
if (!drop && query.sync)
// only if operation is DETACH
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)
@ -385,12 +387,13 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query,
/// Protects from concurrent CREATE TABLE queries
auto db_guard = DatabaseCatalog::instance().getExclusiveDDLGuardForDatabase(database_name);
if (!drop)
// only if operation is DETACH
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/erasing the database.
if (!truncate)
DatabaseCatalog::instance().detachDatabase(getContext(), database_name, drop, database->shouldBeEmptyOnDetach());
}
}

View File

@ -0,0 +1,18 @@
<clickhouse>
<remote_servers>
<test_cluster>
<shard>
<replica>
<host>node1</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>node2</host>
<port>9000</port>
</replica>
</shard>
</test_cluster>
</remote_servers>
</clickhouse>

View File

@ -0,0 +1,21 @@
<clickhouse>
<remote_servers>
<test_cluster>
<shard>
<internal_replication>true</internal_replication>
<replica>
<host>node1</host>
<port>9000</port>
</replica>
<replica>
<host>node2</host>
<port>9000</port>
</replica>
<replica>
<host>node3</host>
<port>9000</port>
</replica>
</shard>
</test_cluster>
</remote_servers>
</clickhouse>

View File

@ -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") == ""

View File

@ -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") == ""

View File

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

View File

@ -0,0 +1,76 @@
-- Tags: no-parallel
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;