Merge pull request #61727 from yariks5s/fix_undrop_query

Revert "Revert "Un-flake test_undrop_query""
This commit is contained in:
Yarik Briukhovetskyi 2024-03-27 13:12:50 +01:00 committed by GitHub
commit 844bb4f06c
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
7 changed files with 50 additions and 62 deletions

View File

@ -945,9 +945,9 @@ Hard limit is configured via system tools
## database_atomic_delay_before_drop_table_sec {#database_atomic_delay_before_drop_table_sec} ## database_atomic_delay_before_drop_table_sec {#database_atomic_delay_before_drop_table_sec}
Sets the delay before remove table data in seconds. If the query has `SYNC` modifier, this setting is ignored. The delay during which a dropped table can be restored using the [UNDROP](/docs/en/sql-reference/statements/undrop.md) statement. If `DROP TABLE` ran with a `SYNC` modifier, the setting is ignored.
Default value: `480` (8 minute). Default value: `480` (8 minutes).
## database_catalog_unused_dir_hide_timeout_sec {#database_catalog_unused_dir_hide_timeout_sec} ## database_catalog_unused_dir_hide_timeout_sec {#database_catalog_unused_dir_hide_timeout_sec}

View File

@ -13,13 +13,6 @@ a system table called `system.dropped_tables`.
If you have a materialized view without a `TO` clause associated with the dropped table, then you will also have to UNDROP the inner table of that view. If you have a materialized view without a `TO` clause associated with the dropped table, then you will also have to UNDROP the inner table of that view.
:::note
UNDROP TABLE is experimental. To use it add this setting:
```sql
set allow_experimental_undrop_table_query = 1;
```
:::
:::tip :::tip
Also see [DROP TABLE](/docs/en/sql-reference/statements/drop.md) Also see [DROP TABLE](/docs/en/sql-reference/statements/drop.md)
::: :::
@ -32,60 +25,53 @@ UNDROP TABLE [db.]name [UUID '<uuid>'] [ON CLUSTER cluster]
**Example** **Example**
``` sql
set allow_experimental_undrop_table_query = 1;
```
```sql ```sql
CREATE TABLE undropMe CREATE TABLE tab
( (
`id` UInt8 `id` UInt8
) )
ENGINE = MergeTree ENGINE = MergeTree
ORDER BY id ORDER BY id;
```
DROP TABLE tab;
```sql
DROP TABLE undropMe
```
```sql
SELECT * SELECT *
FROM system.dropped_tables FROM system.dropped_tables
FORMAT Vertical FORMAT Vertical;
``` ```
```response ```response
Row 1: Row 1:
────── ──────
index: 0 index: 0
database: default database: default
table: undropMe table: tab
uuid: aa696a1a-1d70-4e60-a841-4c80827706cc uuid: aa696a1a-1d70-4e60-a841-4c80827706cc
engine: MergeTree engine: MergeTree
metadata_dropped_path: /var/lib/clickhouse/metadata_dropped/default.undropMe.aa696a1a-1d70-4e60-a841-4c80827706cc.sql metadata_dropped_path: /var/lib/clickhouse/metadata_dropped/default.tab.aa696a1a-1d70-4e60-a841-4c80827706cc.sql
table_dropped_time: 2023-04-05 14:12:12 table_dropped_time: 2023-04-05 14:12:12
1 row in set. Elapsed: 0.001 sec. 1 row in set. Elapsed: 0.001 sec.
``` ```
```sql ```sql
UNDROP TABLE undropMe UNDROP TABLE tab;
```
```response
Ok.
```
```sql
SELECT * SELECT *
FROM system.dropped_tables FROM system.dropped_tables
FORMAT Vertical FORMAT Vertical;
```
```response ```response
Ok. Ok.
0 rows in set. Elapsed: 0.001 sec. 0 rows in set. Elapsed: 0.001 sec.
``` ```
```sql ```sql
DESCRIBE TABLE undropMe DESCRIBE TABLE tab
FORMAT Vertical FORMAT Vertical;
``` ```
```response ```response
Row 1: Row 1:
────── ──────

View File

@ -1143,7 +1143,7 @@ void DatabaseCatalog::dequeueDroppedTableCleanup(StorageID table_id)
TableMarkedAsDropped dropped_table; TableMarkedAsDropped dropped_table;
{ {
std::lock_guard lock(tables_marked_dropped_mutex); std::lock_guard lock(tables_marked_dropped_mutex);
time_t latest_drop_time = std::numeric_limits<time_t>::min(); auto latest_drop_time = std::numeric_limits<time_t>::min();
auto it_dropped_table = tables_marked_dropped.end(); auto it_dropped_table = tables_marked_dropped.end();
for (auto it = tables_marked_dropped.begin(); it != tables_marked_dropped.end(); ++it) for (auto it = tables_marked_dropped.begin(); it != tables_marked_dropped.end(); ++it)
{ {
@ -1168,7 +1168,7 @@ void DatabaseCatalog::dequeueDroppedTableCleanup(StorageID table_id)
} }
if (it_dropped_table == tables_marked_dropped.end()) if (it_dropped_table == tables_marked_dropped.end())
throw Exception(ErrorCodes::UNKNOWN_TABLE, throw Exception(ErrorCodes::UNKNOWN_TABLE,
"The drop task of table {} is in progress, has been dropped or the database engine doesn't support it", "Table {} is being dropped, has been dropped, or the database engine does not support UNDROP",
table_id.getNameForLogs()); table_id.getNameForLogs());
latest_metadata_dropped_path = it_dropped_table->metadata_path; latest_metadata_dropped_path = it_dropped_table->metadata_path;
String table_metadata_path = getPathForMetadata(it_dropped_table->table_id); String table_metadata_path = getPathForMetadata(it_dropped_table->table_id);

View File

@ -18,14 +18,16 @@ namespace ErrorCodes
extern const int SUPPORT_IS_DISABLED; extern const int SUPPORT_IS_DISABLED;
} }
InterpreterUndropQuery::InterpreterUndropQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) InterpreterUndropQuery::InterpreterUndropQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_)
: WithMutableContext(context_)
, query_ptr(query_ptr_)
{ {
} }
BlockIO InterpreterUndropQuery::execute() BlockIO InterpreterUndropQuery::execute()
{ {
getContext()->checkAccess(AccessType::UNDROP_TABLE); getContext()->checkAccess(AccessType::UNDROP_TABLE);
auto & undrop = query_ptr->as<ASTUndropQuery &>(); auto & undrop = query_ptr->as<ASTUndropQuery &>();
if (!undrop.cluster.empty() && !maybeRemoveOnCluster(query_ptr, getContext())) if (!undrop.cluster.empty() && !maybeRemoveOnCluster(query_ptr, getContext()))
{ {

View File

@ -1,3 +1,3 @@
<clickhouse> <clickhouse>
<database_atomic_delay_before_drop_table_sec>3</database_atomic_delay_before_drop_table_sec> <database_atomic_delay_before_drop_table_sec>20</database_atomic_delay_before_drop_table_sec>
</clickhouse> </clickhouse>

View File

@ -1,5 +1,6 @@
import pytest import pytest
import uuid import uuid
import logging
import time import time
from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseCluster
@ -20,29 +21,28 @@ def started_cluster():
def test_undrop_drop_and_undrop_loop(started_cluster): def test_undrop_drop_and_undrop_loop(started_cluster):
# create, drop, undrop, drop, undrop table 5 times uuid_list = []
for _ in range(5):
table_uuid = str(uuid.uuid1()) for i in range(4):
table = f"test_undrop_loop" table_uuid = uuid.uuid1().__str__()
uuid_list.append(table_uuid)
logging.info(f"table_uuid: {table_uuid}")
node.query( node.query(
f"CREATE TABLE {table} " f"CREATE TABLE test_undrop_{i} UUID '{table_uuid}' (id Int32) ENGINE = MergeTree() ORDER BY id;"
f"UUID '{table_uuid}' (id Int32) "
f"Engine=MergeTree() ORDER BY id"
) )
node.query(f"DROP TABLE {table}") node.query(f"DROP TABLE test_undrop_{i};")
node.query(f"UNDROP TABLE {table} UUID '{table_uuid}'")
node.query(f"DROP TABLE {table}") for i in range(4):
# database_atomic_delay_before_drop_table_sec=3 if (
time.sleep(6) i >= 3
): # First 3 tables are undropped after 0, 5 and 10 seconds. Fourth is undropped after 21 seconds
""" time.sleep(6)
Expect two things: error = node.query_and_get_error(
1. Table is dropped - UNKNOWN_TABLE in error f"UNDROP TABLE test_undrop_loop_{i} UUID '{uuid_list[i]}';"
2. Table in process of dropping - Return code: 60. )
The drop task of table ... (uuid) is in progress, assert "UNKNOWN_TABLE" in error
has been dropped or the database engine doesn't support it else:
""" node.query(f"UNDROP TABLE test_undrop_loop_{i} UUID '{uuid_list[i]}';")
error = node.query_and_get_error(f"UNDROP TABLE {table} UUID '{table_uuid}'") time.sleep(5)
assert "UNKNOWN_TABLE" in error or "The drop task of table" in error

View File

@ -17,7 +17,7 @@ drop table if exists 02681_undrop_detach sync;
create table 02681_undrop_detach (id Int32, num Int32) Engine=MergeTree() order by id; create table 02681_undrop_detach (id Int32, num Int32) Engine=MergeTree() order by id;
insert into 02681_undrop_detach values (1, 1); insert into 02681_undrop_detach values (1, 1);
detach table 02681_undrop_detach sync; detach table 02681_undrop_detach sync;
undrop table 02681_undrop_detach; -- { serverError 57 } undrop table 02681_undrop_detach; -- { serverError TABLE_ALREADY_EXISTS }
attach table 02681_undrop_detach; attach table 02681_undrop_detach;
alter table 02681_undrop_detach update num = 2 where id = 1; alter table 02681_undrop_detach update num = 2 where id = 1;
select command from system.mutations where table='02681_undrop_detach' and database=currentDatabase() limit 1; select command from system.mutations where table='02681_undrop_detach' and database=currentDatabase() limit 1;
@ -85,5 +85,5 @@ drop table 02681_undrop_multiple;
select table from system.dropped_tables where table = '02681_undrop_multiple' limit 1; select table from system.dropped_tables where table = '02681_undrop_multiple' limit 1;
undrop table 02681_undrop_multiple; undrop table 02681_undrop_multiple;
select * from 02681_undrop_multiple order by id; select * from 02681_undrop_multiple order by id;
undrop table 02681_undrop_multiple; -- { serverError 57 } undrop table 02681_undrop_multiple; -- { serverError TABLE_ALREADY_EXISTS }
drop table 02681_undrop_multiple sync; drop table 02681_undrop_multiple sync;