mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Mark a setting obsolete (#57336)
* Update Settings.h * Update InterpreterUndropQuery.cpp * Update 02888_obsolete_settings.reference * Update test.py * Update 02681_undrop_query.sql * Update 02681_undrop_query_uuid.sh
This commit is contained in:
parent
0e563e652c
commit
a2dc2d9a53
@ -827,7 +827,6 @@ class IColumn;
|
||||
M(UInt64, grace_hash_join_initial_buckets, 1, "Initial number of grace hash join buckets", 0) \
|
||||
M(UInt64, grace_hash_join_max_buckets, 1024, "Limit on the number of grace hash join buckets", 0) \
|
||||
M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \
|
||||
M(Bool, allow_experimental_undrop_table_query, true, "Allow to use undrop query to restore dropped table in a limited time", 0) \
|
||||
M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \
|
||||
M(UInt64, extract_kvp_max_pairs_per_row, 1000, "Max number pairs that can be produced by extractKeyValuePairs function. Used to safeguard against consuming too much memory.", 0) \
|
||||
M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \
|
||||
@ -898,6 +897,7 @@ class IColumn;
|
||||
MAKE_OBSOLETE(M, Bool, query_plan_optimize_projection, true) \
|
||||
MAKE_OBSOLETE(M, Bool, query_cache_store_results_of_queries_with_nondeterministic_functions, false) \
|
||||
MAKE_OBSOLETE(M, Bool, optimize_move_functions_out_of_any, false) \
|
||||
MAKE_OBSOLETE(M, Bool, allow_experimental_undrop_table_query, true) \
|
||||
|
||||
/** The section above is for obsolete settings. Do not add anything there. */
|
||||
|
||||
|
@ -23,11 +23,6 @@ InterpreterUndropQuery::InterpreterUndropQuery(const ASTPtr & query_ptr_, Contex
|
||||
|
||||
BlockIO InterpreterUndropQuery::execute()
|
||||
{
|
||||
if (!getContext()->getSettingsRef().allow_experimental_undrop_table_query)
|
||||
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
|
||||
"Undrop table is experimental. "
|
||||
"Set `allow_experimental_undrop_table_query` setting to enable it");
|
||||
|
||||
getContext()->checkAccess(AccessType::UNDROP_TABLE);
|
||||
auto & undrop = query_ptr->as<ASTUndropQuery &>();
|
||||
if (!undrop.cluster.empty() && !maybeRemoveOnCluster(query_ptr, getContext()))
|
||||
|
@ -23,7 +23,6 @@ def started_cluster():
|
||||
|
||||
def test_undrop_drop_and_undrop_loop(started_cluster):
|
||||
count = 0
|
||||
node.query("set allow_experimental_undrop_table_query = 1;")
|
||||
while count < 10:
|
||||
random_sec = random.randint(0, 10)
|
||||
table_uuid = uuid.uuid1().__str__()
|
||||
@ -45,7 +44,7 @@ def test_undrop_drop_and_undrop_loop(started_cluster):
|
||||
+ count.__str__()
|
||||
+ " uuid '"
|
||||
+ table_uuid
|
||||
+ "' settings allow_experimental_undrop_table_query = 1;"
|
||||
+ "';"
|
||||
)
|
||||
assert "UNKNOWN_TABLE" in error
|
||||
else:
|
||||
@ -54,6 +53,6 @@ def test_undrop_drop_and_undrop_loop(started_cluster):
|
||||
+ count.__str__()
|
||||
+ " uuid '"
|
||||
+ table_uuid
|
||||
+ "' settings allow_experimental_undrop_table_query = 1;"
|
||||
+ "';"
|
||||
)
|
||||
count = count + 1
|
||||
|
@ -1,7 +1,6 @@
|
||||
-- Tags: no-ordinary-database, no-replicated-database, distributed, zookeeper
|
||||
|
||||
set database_atomic_wait_for_drop_and_detach_synchronously = 0;
|
||||
set allow_experimental_undrop_table_query = 1;
|
||||
|
||||
select 'test MergeTree undrop';
|
||||
drop table if exists 02681_undrop_mergetree sync;
|
||||
|
@ -13,7 +13,7 @@ ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -q "create table 02681_u
|
||||
${CLICKHOUSE_CLIENT} -q "insert into 02681_undrop_uuid values (1),(2),(3);"
|
||||
${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -q "drop table 02681_undrop_uuid on cluster test_shard_localhost settings database_atomic_wait_for_drop_and_detach_synchronously = 0;"
|
||||
${CLICKHOUSE_CLIENT} -q "select table from system.dropped_tables where table = '02681_undrop_uuid' limit 1;"
|
||||
${CLICKHOUSE_CLIENT} -q "undrop table 02681_undrop_uuid UUID '$uuid2' settings allow_experimental_undrop_table_query = 1;" 2>&1| grep -Faq "UNKNOWN_TABLE" && echo OK
|
||||
${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -q "undrop table 02681_undrop_uuid UUID '$uuid' on cluster test_shard_localhost settings allow_experimental_undrop_table_query = 1;"
|
||||
${CLICKHOUSE_CLIENT} -q "undrop table 02681_undrop_uuid UUID '$uuid2';" 2>&1| grep -Faq "UNKNOWN_TABLE" && echo OK
|
||||
${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -q "undrop table 02681_undrop_uuid UUID '$uuid' on cluster test_shard_localhost;"
|
||||
${CLICKHOUSE_CLIENT} -q "select * from 02681_undrop_uuid order by id;"
|
||||
${CLICKHOUSE_CLIENT} -q "drop table 02681_undrop_uuid sync;"
|
||||
|
@ -5,6 +5,7 @@ allow_experimental_database_atomic
|
||||
allow_experimental_geo_types
|
||||
allow_experimental_map_type
|
||||
allow_experimental_query_cache
|
||||
allow_experimental_undrop_table_query
|
||||
allow_experimental_window_functions
|
||||
async_insert_cleanup_timeout_ms
|
||||
async_insert_stale_timeout_ms
|
||||
|
Loading…
Reference in New Issue
Block a user