mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Merge pull request #56942 from shabroo/master
Emit a warning in clickhouse-client if Ordinary database engine is used
This commit is contained in:
commit
4fd64a696b
@ -513,6 +513,9 @@ void registerDatabaseOrdinary(DatabaseFactory & factory)
|
||||
throw Exception(
|
||||
ErrorCodes::UNKNOWN_DATABASE_ENGINE,
|
||||
"Ordinary database engine is deprecated (see also allow_deprecated_database_ordinary setting)");
|
||||
|
||||
args.context->addWarningMessageAboutDatabaseOrdinary(args.database_name);
|
||||
|
||||
return make_shared<DatabaseOrdinary>(
|
||||
args.database_name,
|
||||
args.metadata_path,
|
||||
|
@ -724,7 +724,7 @@ struct ContextSharedPart : boost::noncopyable
|
||||
void addWarningMessage(const String & message) TSA_REQUIRES(mutex)
|
||||
{
|
||||
/// A warning goes both: into server's log; stored to be placed in `system.warnings` table.
|
||||
log->warning(message);
|
||||
LOG_WARNING(log, "{}", message);
|
||||
warnings.push_back(message);
|
||||
}
|
||||
|
||||
@ -1177,6 +1177,29 @@ void Context::addWarningMessage(const String & msg) const
|
||||
shared->addWarningMessage(msg);
|
||||
}
|
||||
|
||||
void Context::addWarningMessageAboutDatabaseOrdinary(const String & database_name) const
|
||||
{
|
||||
std::lock_guard lock(shared->mutex);
|
||||
|
||||
/// We would like to report only about the first database with engine Ordinary
|
||||
static std::atomic_bool is_called = false;
|
||||
if (is_called.exchange(true))
|
||||
return;
|
||||
|
||||
auto suppress_re = shared->getConfigRefWithLock(lock).getString("warning_supress_regexp", "");
|
||||
/// We don't use getFlagsPath method, because it takes a shared lock.
|
||||
auto convert_databases_flag = fs::path(shared->flags_path) / "convert_ordinary_to_atomic";
|
||||
auto message = fmt::format("Server has databases (for example `{}`) with Ordinary engine, which was deprecated. "
|
||||
"To convert this database to a new Atomic engine, please create a forcing flag {} and make sure that ClickHouse has write permission for it. "
|
||||
"Example: sudo touch '{}' && sudo chmod 666 '{}'",
|
||||
database_name,
|
||||
convert_databases_flag.string(), convert_databases_flag.string(), convert_databases_flag.string());
|
||||
|
||||
bool is_supressed = !suppress_re.empty() && re2::RE2::PartialMatch(message, suppress_re);
|
||||
if (!is_supressed)
|
||||
shared->addWarningMessage(message);
|
||||
}
|
||||
|
||||
void Context::setConfig(const ConfigurationPtr & config)
|
||||
{
|
||||
shared->setConfig(config);
|
||||
|
@ -536,6 +536,7 @@ public:
|
||||
void setUserScriptsPath(const String & path);
|
||||
|
||||
void addWarningMessage(const String & msg) const;
|
||||
void addWarningMessageAboutDatabaseOrdinary(const String & database_name) const;
|
||||
|
||||
void setTemporaryStorageInCache(const String & cache_disk_name, size_t max_size);
|
||||
void setTemporaryStoragePolicy(const String & policy_name, size_t max_size);
|
||||
|
@ -1,8 +1,9 @@
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
SET allow_experimental_analyzer = 0;
|
||||
SET allow_experimental_window_view = 1;
|
||||
DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier};
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
-- Creation of a database with Ordinary engine emits a warning.
|
||||
CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier} ENGINE=Ordinary;
|
||||
|
||||
DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.mt;
|
||||
|
@ -1,9 +1,11 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
SET allow_experimental_analyzer = 0;
|
||||
SET allow_experimental_window_view = 1;
|
||||
DROP DATABASE IF EXISTS test_01048;
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
-- Creation of a database with Ordinary engine emits a warning.
|
||||
CREATE DATABASE test_01048 ENGINE=Ordinary;
|
||||
|
||||
DROP TABLE IF EXISTS test_01048.mt;
|
||||
|
@ -1,6 +1,8 @@
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier};
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
-- Creation of a database with Ordinary engine emits a warning.
|
||||
CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier} ENGINE=Ordinary; -- Different inner table name with Atomic
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
|
@ -1,9 +1,11 @@
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
SET allow_experimental_analyzer = 0;
|
||||
SET allow_experimental_window_view = 1;
|
||||
|
||||
DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier};
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
-- Creation of a database with Ordinary engine emits a warning.
|
||||
CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier} ENGINE=Ordinary;
|
||||
|
||||
DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.mt;
|
||||
|
@ -1,6 +1,9 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-parallel
|
||||
|
||||
# Creation of a database with Ordinary engine emits a warning.
|
||||
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
@ -1,4 +1,5 @@
|
||||
-- Tags: no-parallel
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP DATABASE IF EXISTS test_01109;
|
||||
CREATE DATABASE test_01109 ENGINE=Atomic;
|
||||
@ -31,6 +32,7 @@ DROP DATABASE IF EXISTS test_01109_other_atomic;
|
||||
DROP DATABASE IF EXISTS test_01109_ordinary;
|
||||
CREATE DATABASE test_01109_other_atomic;
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
-- Creation of a database with Ordinary engine emits a warning.
|
||||
CREATE DATABASE test_01109_ordinary ENGINE=Ordinary;
|
||||
|
||||
CREATE TABLE test_01109_other_atomic.t3 ENGINE=MergeTree() ORDER BY tuple()
|
||||
@ -63,6 +65,3 @@ DROP DATABASE test_01109;
|
||||
DROP DATABASE test_01109_other_atomic;
|
||||
DROP DATABASE test_01109_ordinary;
|
||||
DROP DATABASE test_01109_rename_exists;
|
||||
|
||||
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
2
|
||||
1
|
||||
CREATE DATABASE test_01114_1\nENGINE = Atomic
|
||||
CREATE DATABASE test_01114_2\nENGINE = Atomic
|
||||
CREATE DATABASE test_01114_3\nENGINE = Ordinary
|
||||
|
@ -2,6 +2,9 @@
|
||||
# Tags: no-parallel, no-fasttest
|
||||
# Tag no-fasttest: 45 seconds running
|
||||
|
||||
# Creation of a database with Ordinary engine emits a warning.
|
||||
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: zookeeper, no-replicated-database, no-parallel, no-ordinary-database
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP TABLE IF EXISTS rmt;
|
||||
DROP TABLE IF EXISTS rmt1;
|
||||
DROP TABLE IF EXISTS rmt2;
|
||||
@ -32,6 +34,7 @@ SHOW CREATE TABLE test_01148_atomic.rmt3;
|
||||
|
||||
DROP DATABASE IF EXISTS test_01148_ordinary;
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
-- Creation of a database with Ordinary engine emits a warning.
|
||||
CREATE DATABASE test_01148_ordinary ENGINE=Ordinary;
|
||||
RENAME TABLE test_01148_atomic.rmt3 to test_01148_ordinary.rmt3; -- { serverError 48 }
|
||||
DROP DATABASE test_01148_ordinary;
|
||||
|
@ -1,11 +1,13 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
SET prefer_localhost_replica = 1;
|
||||
|
||||
DROP DATABASE IF EXISTS test_01155_ordinary;
|
||||
DROP DATABASE IF EXISTS test_01155_atomic;
|
||||
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
-- Creation of a database with Ordinary engine emits a warning.
|
||||
CREATE DATABASE test_01155_ordinary ENGINE=Ordinary;
|
||||
CREATE DATABASE test_01155_atomic ENGINE=Atomic;
|
||||
|
||||
@ -70,7 +72,10 @@ RENAME DATABASE test_01155_ordinary TO test_01155_atomic;
|
||||
SET check_table_dependencies=1;
|
||||
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
-- Creation of a database with Ordinary engine emits a warning.
|
||||
SET send_logs_level='fatal';
|
||||
CREATE DATABASE test_01155_ordinary ENGINE=Ordinary;
|
||||
SET send_logs_level='warning';
|
||||
SHOW CREATE DATABASE test_01155_atomic;
|
||||
|
||||
RENAME TABLE test_01155_atomic.mv1 TO test_01155_ordinary.mv1;
|
||||
|
@ -1,6 +1,9 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: zookeeper, no-parallel, no-fasttest
|
||||
|
||||
# Creation of a database with Ordinary engine emits a warning.
|
||||
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
@ -1,8 +1,11 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP DATABASE IF EXISTS dict_db_01224;
|
||||
DROP DATABASE IF EXISTS dict_db_01224_dictionary;
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
-- Creation of a database with Ordinary engine emits a warning.
|
||||
CREATE DATABASE dict_db_01224 ENGINE=Ordinary; -- Different internal dictionary name with Atomic
|
||||
CREATE DATABASE dict_db_01224_dictionary Engine=Dictionary;
|
||||
|
||||
|
@ -1,8 +1,11 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP DATABASE IF EXISTS dict_db_01225;
|
||||
DROP DATABASE IF EXISTS dict_db_01225_dictionary;
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
-- Creation of a database with Ordinary engine emits a warning.
|
||||
CREATE DATABASE dict_db_01225 ENGINE=Ordinary; -- Different internal dictionary name with Atomic
|
||||
CREATE DATABASE dict_db_01225_dictionary Engine=Dictionary;
|
||||
|
||||
|
@ -1,7 +1,10 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP DATABASE IF EXISTS test_01249;
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
-- Creation of a database with Ordinary engine emits a warning.
|
||||
CREATE DATABASE test_01249 ENGINE=Ordinary; -- Full ATTACH requires UUID with Atomic
|
||||
USE test_01249;
|
||||
|
||||
|
@ -1,6 +1,9 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: race, zookeeper, no-parallel
|
||||
|
||||
# Creation of a database with Ordinary engine emits a warning.
|
||||
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
@ -1,7 +1,10 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP DATABASE IF EXISTS test_01516;
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
-- Creation of a database with Ordinary engine emits a warning.
|
||||
CREATE DATABASE test_01516 ENGINE=Ordinary; -- Full ATTACH requires UUID with Atomic
|
||||
USE test_01516;
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
--
|
||||
-- Atomic no SYNC
|
||||
-- (should go first to check that thread for DROP TABLE does not hang)
|
||||
@ -33,6 +35,7 @@ show tables from db_01517_atomic_sync;
|
||||
---
|
||||
drop database if exists db_01517_ordinary;
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
-- Creation of a database with Ordinary engine emits a warning.
|
||||
create database db_01517_ordinary Engine=Ordinary;
|
||||
|
||||
create table db_01517_ordinary.source (key Int) engine=Null;
|
||||
|
@ -1,5 +1,8 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
# Creation of a database with Ordinary engine emits a warning.
|
||||
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
@ -1,6 +1,9 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-parallel
|
||||
|
||||
# Creation of a database with Ordinary engine emits a warning.
|
||||
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
SELECT 'database atomic tests';
|
||||
|
||||
DROP DATABASE IF EXISTS test1601_detach_permanently_atomic;
|
||||
@ -73,6 +75,7 @@ SELECT 'database ordinary tests';
|
||||
|
||||
DROP DATABASE IF EXISTS test1601_detach_permanently_ordinary;
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
-- Creation of a database with Ordinary engine emits a warning.
|
||||
CREATE DATABASE test1601_detach_permanently_ordinary Engine=Ordinary;
|
||||
|
||||
create table test1601_detach_permanently_ordinary.test_name_reuse (number UInt64) engine=MergeTree order by tuple();
|
||||
|
@ -1,7 +1,10 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP database IF EXISTS test_1603_rename_bug_ordinary;
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
-- Creation of a database with Ordinary engine emits a warning.
|
||||
create database test_1603_rename_bug_ordinary engine=Ordinary;
|
||||
create table test_1603_rename_bug_ordinary.foo engine=Memory as select * from numbers(100);
|
||||
create table test_1603_rename_bug_ordinary.bar engine=Log as select * from numbers(200);
|
||||
|
@ -7,6 +7,9 @@
|
||||
# and we can do it compatible with parallel run only in .sh
|
||||
# (via $CLICKHOUSE_DATABASE)
|
||||
|
||||
# Creation of a database with Ordinary engine emits a warning.
|
||||
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
@ -1,5 +1,7 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
@ -1,8 +1,9 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
drop database if exists db_hang;
|
||||
drop database if exists db_hang_temp;
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
-- Creation of a database with Ordinary engine emits a warning.
|
||||
create database db_hang engine=Ordinary;
|
||||
use db_hang;
|
||||
create table db_hang.test(A Int64) Engine=MergeTree order by A;
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
DROP DATABASE IF EXISTS 02265_atomic_db;
|
||||
DROP DATABASE IF EXISTS 02265_ordinary_db;
|
||||
|
@ -6,6 +6,9 @@
|
||||
# In other words to ensure that after WRITE lock failure (DROP),
|
||||
# READ lock (SELECT) available instantly.
|
||||
|
||||
# Creation of a database with Ordinary engine emits a warning.
|
||||
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
@ -3,6 +3,9 @@
|
||||
# Tag no-fasttest: requires S3
|
||||
# Tag no-random-settings, no-random-merge-tree-settings: to avoid creating extra files like serialization.json, this test too exocit anyway
|
||||
|
||||
# Creation of a database with Ordinary engine emits a warning.
|
||||
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
@ -3,6 +3,9 @@
|
||||
# Tag no-fasttest: requires S3
|
||||
# Tag no-random-settings, no-random-merge-tree-settings: to avoid creating extra files like serialization.json, this test too exocit anyway
|
||||
|
||||
# Creation of a database with Ordinary engine emits a warning.
|
||||
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
@ -0,0 +1 @@
|
||||
Ok.
|
@ -0,0 +1,10 @@
|
||||
DROP DATABASE IF EXISTS 02988_ordinary;
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
SET allow_deprecated_database_ordinary = 1;
|
||||
-- Creation of a database with Ordinary engine emits a warning.
|
||||
CREATE DATABASE 02988_ordinary ENGINE=Ordinary;
|
||||
|
||||
SELECT DISTINCT 'Ok.' FROM system.warnings WHERE message ILIKE '%Ordinary%' and message ILIKE '%deprecated%';
|
||||
|
||||
DROP DATABASE IF EXISTS 02988_ordinary;
|
Loading…
Reference in New Issue
Block a user