fix tests

This commit is contained in:
Alexander Tokmakov 2022-06-23 21:19:07 +02:00
parent fb249aca62
commit 0d304f7b8c
22 changed files with 24 additions and 18 deletions

View File

@ -20,7 +20,7 @@ def start_cluster():
try:
cluster.start()
node1.query(
"CREATE DATABASE zktest ENGINE=Ordinary;"
"CREATE DATABASE zktest ENGINE=Ordinary;", settings={"allow_deprecated_database_ordinary": 1}
) # Different behaviour with Atomic
node1.query(
"""

View File

@ -34,7 +34,7 @@ def create_force_drop_flag(node):
@pytest.mark.parametrize("engine", ["Ordinary", "Atomic"])
def test_attach_partition_with_large_destination(started_cluster, engine):
# Initialize
node.query("CREATE DATABASE db ENGINE={}".format(engine))
node.query("CREATE DATABASE db ENGINE={}".format(engine), settings={"allow_deprecated_database_ordinary": 1})
node.query(
"CREATE TABLE db.destination (n UInt64) ENGINE=ReplicatedMergeTree('/test/destination', 'r1') ORDER BY n PARTITION BY n % 2"
)

View File

@ -15,7 +15,7 @@ def started_cluster():
try:
cluster.start()
instance.query(
"CREATE DATABASE test ENGINE = Ordinary"
"CREATE DATABASE test ENGINE = Ordinary", settings={"allow_deprecated_database_ordinary": 1}
) # Different path in shadow/ with Atomic
instance.query("DROP TABLE IF EXISTS test.tbl")
instance.query(

View File

@ -202,7 +202,7 @@ def test_backup_from_old_version_config(started_cluster):
def test_backup_and_alter(started_cluster):
node4.query(
"CREATE DATABASE test ENGINE=Ordinary"
"CREATE DATABASE test ENGINE=Ordinary", settings={"allow_deprecated_database_ordinary": 1}
) # Different path in shadow/ with Atomic
node4.query(

View File

@ -16,7 +16,7 @@ def start_cluster():
for node in nodes:
node.query("CREATE DATABASE IF NOT EXISTS test")
# Different internal dictionary name with Atomic
node.query("CREATE DATABASE IF NOT EXISTS test_ordinary ENGINE=Ordinary")
node.query("CREATE DATABASE IF NOT EXISTS test_ordinary ENGINE=Ordinary", settings={"allow_deprecated_database_ordinary": 1})
node.query("CREATE DATABASE IF NOT EXISTS atest")
node.query("CREATE DATABASE IF NOT EXISTS ztest")
node.query("CREATE TABLE test.source(x UInt64, y UInt64) ENGINE=Log")

View File

@ -552,7 +552,7 @@ def test_replicated_without_arguments(test_cluster):
)
test_cluster.ddl_check_query(
instance, "CREATE DATABASE test_ordinary ON CLUSTER cluster ENGINE=Ordinary"
instance, "CREATE DATABASE test_ordinary ON CLUSTER cluster ENGINE=Ordinary", settings={"allow_deprecated_database_ordinary": 1}
)
assert (
"are supported only for ON CLUSTER queries with Atomic database engine"

View File

@ -20,7 +20,7 @@ def start_cluster():
try:
cluster.start()
node.query(
"CREATE DATABASE IF NOT EXISTS test ENGINE=Ordinary"
"CREATE DATABASE IF NOT EXISTS test ENGINE=Ordinary", settings={"allow_deprecated_database_ordinary": 1}
) # Different paths with Atomic
yield cluster
finally:

View File

@ -16,7 +16,7 @@ def started_cluster():
def test_file_path_escaping(started_cluster):
node.query("CREATE DATABASE IF NOT EXISTS test ENGINE = Ordinary")
node.query("CREATE DATABASE IF NOT EXISTS test ENGINE = Ordinary", settings={"allow_deprecated_database_ordinary": 1})
node.query(
"""
CREATE TABLE test.`T.a_b,l-e!` (`~Id` UInt32)

View File

@ -33,7 +33,7 @@ def create_force_drop_flag(node):
@pytest.mark.parametrize("engine", ["Ordinary", "Atomic"])
def test_drop_materialized_view(started_cluster, engine):
node.query("CREATE DATABASE d ENGINE={}".format(engine))
node.query("CREATE DATABASE d ENGINE={}".format(engine), settings={"allow_deprecated_database_ordinary": 1})
node.query(
"CREATE TABLE d.rmt (n UInt64) ENGINE=ReplicatedMergeTree('/test/rmt', 'r1') ORDER BY n PARTITION BY n % 2"
)

View File

@ -95,7 +95,7 @@ def test_blocade_leader(started_cluster):
wait_nodes()
try:
for i, node in enumerate([node1, node2, node3]):
node.query("CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary")
node.query("CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary", settings={"allow_deprecated_database_ordinary": 1})
node.query(
"CREATE TABLE IF NOT EXISTS ordinary.t1 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t1', '{}') ORDER BY tuple()".format(
i + 1
@ -296,7 +296,7 @@ def test_blocade_leader_twice(started_cluster):
wait_nodes()
try:
for i, node in enumerate([node1, node2, node3]):
node.query("CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary")
node.query("CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary", settings={"allow_deprecated_database_ordinary": 1})
node.query(
"CREATE TABLE IF NOT EXISTS ordinary.t2 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t2', '{}') ORDER BY tuple()".format(
i + 1

View File

@ -93,7 +93,7 @@ def create_table(
node.query(
"CREATE DATABASE IF NOT EXISTS s3 ENGINE = {engine}".format(
engine="Atomic" if db_atomic else "Ordinary"
)
), settings={"allow_deprecated_database_ordinary": 1}
)
create_table_statement = """

View File

@ -14,7 +14,7 @@ def started_cluster():
try:
cluster.start()
q(
"CREATE DATABASE test ENGINE = Ordinary"
"CREATE DATABASE test ENGINE = Ordinary", settings={"allow_deprecated_database_ordinary": 1}
) # Different path in shadow/ with Atomic
yield cluster

View File

@ -720,7 +720,7 @@ def test_in_memory_alters(start_cluster):
def test_polymorphic_parts_index(start_cluster):
node1.query(
"CREATE DATABASE test_index ENGINE=Ordinary"
"CREATE DATABASE test_index ENGINE=Ordinary", settings={"allow_deprecated_database_ordinary": 1}
) # Different paths with Atomic
node1.query(
"""

View File

@ -52,7 +52,7 @@ def test_replicated_merge_tree_defaults_compatibility(started_cluster):
"""
for node in (node1, node2):
node.query("CREATE DATABASE test ENGINE = Ordinary")
node.query("CREATE DATABASE test ENGINE = Ordinary", settings={"allow_deprecated_database_ordinary": 1})
node.query(create_query.format(replica=node.name))
node1.query("DETACH TABLE test.table")

View File

@ -76,7 +76,7 @@ def create_table(node, table_name, schema, attach=False, db_atomic=False, uuid="
"CREATE DATABASE IF NOT EXISTS s3 {on_cluster} ENGINE = {engine}".format(
engine="Atomic" if db_atomic else "Ordinary",
on_cluster="ON CLUSTER '{cluster}'",
)
), settings={"allow_deprecated_database_ordinary": 1}
)
create_table_statement = """

View File

@ -26,9 +26,9 @@ def started_cluster():
try:
cluster.start()
node1.query(
"CREATE DATABASE test ENGINE=Ordinary"
"CREATE DATABASE test ENGINE=Ordinary", settings={"allow_deprecated_database_ordinary": 1}
) # Different paths with Atomic
node2.query("CREATE DATABASE test ENGINE=Ordinary")
node2.query("CREATE DATABASE test ENGINE=Ordinary", settings={"allow_deprecated_database_ordinary": 1})
yield cluster
finally:

View File

@ -1,6 +1,7 @@
<test>
<settings>
<max_threads>5</max_threads>
<allow_deprecated_syntax_for_merge_tree>1</allow_deprecated_syntax_for_merge_tree>
</settings>
<!-- Create four empty tables. We will test select from Merge table involving one real table and four empty tables.

View File

@ -10,6 +10,7 @@ set -o pipefail
echo "
DROP TABLE IF EXISTS two_blocks;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE two_blocks (d Date) ENGINE = MergeTree(d, d, 1);
INSERT INTO two_blocks VALUES ('2000-01-01');
INSERT INTO two_blocks VALUES ('2000-01-02');

View File

@ -2,6 +2,7 @@
-- Tag no-replicated-database: Old syntax is not allowed
DROP TABLE IF EXISTS primary_key;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE primary_key (d Date DEFAULT today(), x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00215/primary_key', 'r1', d, -x, 1);
INSERT INTO primary_key (x) VALUES (1), (2), (3);

View File

@ -2,6 +2,7 @@ DROP TABLE IF EXISTS secure1;
DROP TABLE IF EXISTS secure2;
DROP TABLE IF EXISTS secure3;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE secure1 ( date Date, a Int32, b Int32, c Int32, d Int32) ENGINE = MergeTree(date, (a, date), 8192);
CREATE TABLE secure2 ( date Date, a Int32, b Int32, c Int32, d Int32) ENGINE = Distributed(test_shard_localhost_secure, currentDatabase(), 'secure1');
CREATE TABLE secure3 ( date Date, a Int32, b Int32, c Int32, d Int32) ENGINE = Distributed(test_shard_localhost_secure, currentDatabase(), 'secure2');

View File

@ -7,6 +7,7 @@ DROP TABLE IF EXISTS dict_string;
DROP TABLE IF EXISTS dict_ui64;
DROP TABLE IF EXISTS video_views;
set allow_deprecated_syntax_for_merge_tree=1;
CREATE TABLE video_views
(
entityIri String,

View File

@ -8,6 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
$CLICKHOUSE_CLIENT -n --query="
set allow_deprecated_database_ordinary=1;
DROP DATABASE IF EXISTS 01280_db;
CREATE DATABASE 01280_db Engine = Ordinary;
DROP TABLE IF EXISTS 01280_db.table_for_dict;