fix tests

This commit is contained in:
Alexander Tokmakov 2020-09-22 00:09:50 +03:00
parent 13e41cdd20
commit 91cbd49e85
14 changed files with 216 additions and 185 deletions

View File

@ -858,11 +858,9 @@ BlockIO InterpreterCreateQuery::execute()
auto & create = query_ptr->as<ASTCreateQuery &>(); auto & create = query_ptr->as<ASTCreateQuery &>();
if (!create.cluster.empty()) if (!create.cluster.empty())
{ {
/// Allows to execute ON CLUSTER queries during version upgrade
bool force_backward_compatibility = !context.getSettingsRef().show_table_uuid_in_table_create_query_if_not_nil;
/// For CREATE query generate UUID on initiator, so it will be the same on all hosts. /// For CREATE query generate UUID on initiator, so it will be the same on all hosts.
/// It will be ignored if database does not support UUIDs. /// It will be ignored if database does not support UUIDs.
if (!force_backward_compatibility && !create.attach && create.uuid == UUIDHelpers::Nil) if (!create.attach && create.uuid == UUIDHelpers::Nil)
create.uuid = UUIDHelpers::generateV4(); create.uuid = UUIDHelpers::generateV4();
return executeDDLQueryOnCluster(query_ptr, context, getRequiredAccess()); return executeDDLQueryOnCluster(query_ptr, context, getRequiredAccess());
} }

View File

@ -8,6 +8,7 @@
#include <Parsers/ASTExpressionList.h> #include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Common/ZooKeeper/ZooKeeper.h> #include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
@ -42,7 +43,7 @@ NamesAndTypesList StorageSystemZooKeeper::getNamesAndTypes()
} }
static bool extractPathImpl(const IAST & elem, String & res) static bool extractPathImpl(const IAST & elem, String & res, const Context & context)
{ {
const auto * function = elem.as<ASTFunction>(); const auto * function = elem.as<ASTFunction>();
if (!function) if (!function)
@ -51,7 +52,7 @@ static bool extractPathImpl(const IAST & elem, String & res)
if (function->name == "and") if (function->name == "and")
{ {
for (const auto & child : function->arguments->children) for (const auto & child : function->arguments->children)
if (extractPathImpl(*child, res)) if (extractPathImpl(*child, res, context))
return true; return true;
return false; return false;
@ -60,23 +61,24 @@ static bool extractPathImpl(const IAST & elem, String & res)
if (function->name == "equals") if (function->name == "equals")
{ {
const auto & args = function->arguments->as<ASTExpressionList &>(); const auto & args = function->arguments->as<ASTExpressionList &>();
const IAST * value; ASTPtr value;
if (args.children.size() != 2) if (args.children.size() != 2)
return false; return false;
const ASTIdentifier * ident; const ASTIdentifier * ident;
if ((ident = args.children.at(0)->as<ASTIdentifier>())) if ((ident = args.children.at(0)->as<ASTIdentifier>()))
value = args.children.at(1).get(); value = args.children.at(1);
else if ((ident = args.children.at(1)->as<ASTIdentifier>())) else if ((ident = args.children.at(1)->as<ASTIdentifier>()))
value = args.children.at(0).get(); value = args.children.at(0);
else else
return false; return false;
if (ident->name != "path") if (ident->name != "path")
return false; return false;
const auto * literal = value->as<ASTLiteral>(); auto evaluated = evaluateConstantExpressionAsLiteral(value, context);
const auto * literal = evaluated->as<ASTLiteral>();
if (!literal) if (!literal)
return false; return false;
@ -93,20 +95,20 @@ static bool extractPathImpl(const IAST & elem, String & res)
/** Retrieve from the query a condition of the form `path = 'path'`, from conjunctions in the WHERE clause. /** Retrieve from the query a condition of the form `path = 'path'`, from conjunctions in the WHERE clause.
*/ */
static String extractPath(const ASTPtr & query) static String extractPath(const ASTPtr & query, const Context & context)
{ {
const auto & select = query->as<ASTSelectQuery &>(); const auto & select = query->as<ASTSelectQuery &>();
if (!select.where()) if (!select.where())
return ""; return "";
String res; String res;
return extractPathImpl(*select.where(), res) ? res : ""; return extractPathImpl(*select.where(), res, context) ? res : "";
} }
void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const
{ {
String path = extractPath(query_info.query); String path = extractPath(query_info.query, context);
if (path.empty()) if (path.empty())
throw Exception("SELECT from system.zookeeper table must contain condition like path = 'path' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); throw Exception("SELECT from system.zookeeper table must contain condition like path = 'path' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS);

View File

@ -17,6 +17,7 @@ node4 = cluster.add_instance('node4')
def started_cluster(): def started_cluster():
try: try:
cluster.start() cluster.start()
yield cluster yield cluster
finally: finally:
cluster.shutdown() cluster.shutdown()
@ -141,22 +142,24 @@ def test_backup_from_old_version_config(started_cluster):
def test_backup_and_alter(started_cluster): def test_backup_and_alter(started_cluster):
node4.query("CREATE TABLE backup_table(A Int64, B String, C Date) Engine = MergeTree order by tuple()") node4.query("CREATE DATABASE test ENGINE=Ordinary")
node4.query("INSERT INTO backup_table VALUES(2, '2', toDate('2019-10-01'))") node4.query("CREATE TABLE test.backup_table(A Int64, B String, C Date) Engine = MergeTree order by tuple()")
node4.query("ALTER TABLE backup_table FREEZE PARTITION tuple();") node4.query("INSERT INTO test.backup_table VALUES(2, '2', toDate('2019-10-01'))")
node4.query("ALTER TABLE backup_table DROP COLUMN C") node4.query("ALTER TABLE test.backup_table FREEZE PARTITION tuple();")
node4.query("ALTER TABLE backup_table MODIFY COLUMN B UInt64") node4.query("ALTER TABLE test.backup_table DROP COLUMN C")
node4.query("ALTER TABLE backup_table DROP PARTITION tuple()") node4.query("ALTER TABLE test.backup_table MODIFY COLUMN B UInt64")
node4.query("ALTER TABLE test.backup_table DROP PARTITION tuple()")
node4.exec_in_container(['bash', '-c', node4.exec_in_container(['bash', '-c',
'cp -r /var/lib/clickhouse/shadow/1/data/default/backup_table/all_1_1_0/ /var/lib/clickhouse/data/default/backup_table/detached']) 'cp -r /var/lib/clickhouse/shadow/1/data/test/backup_table/all_1_1_0/ /var/lib/clickhouse/data/test/backup_table/detached'])
node4.query("ALTER TABLE backup_table ATTACH PARTITION tuple()") node4.query("ALTER TABLE test.backup_table ATTACH PARTITION tuple()")
assert node4.query("SELECT sum(A) FROM backup_table") == "2\n" assert node4.query("SELECT sum(A) FROM test.backup_table") == "2\n"
assert node4.query("SELECT B + 2 FROM backup_table") == "4\n" assert node4.query("SELECT B + 2 FROM test.backup_table") == "4\n"

View File

@ -326,19 +326,15 @@ def test_socket_timeout(test_cluster):
def test_replicated_without_arguments(test_cluster): def test_replicated_without_arguments(test_cluster):
rules = test_cluster.pm_random_drops.pop_rules() rules = test_cluster.pm_random_drops.pop_rules()
instance = test_cluster.instances['ch1'] instance = test_cluster.instances['ch1']
test_cluster.ddl_check_query(instance, "CREATE DATABASE test_atomic ON CLUSTER cluster ENGINE=Atomic", test_cluster.ddl_check_query(instance, "CREATE DATABASE test_atomic ON CLUSTER cluster ENGINE=Atomic")
settings={'show_table_uuid_in_table_create_query_if_not_nil': 1})
test_cluster.ddl_check_query(instance, test_cluster.ddl_check_query(instance,
"CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n", "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n")
settings={'show_table_uuid_in_table_create_query_if_not_nil': 1})
test_cluster.ddl_check_query(instance, "DROP TABLE test_atomic.rmt ON CLUSTER cluster") test_cluster.ddl_check_query(instance, "DROP TABLE test_atomic.rmt ON CLUSTER cluster")
test_cluster.ddl_check_query(instance, test_cluster.ddl_check_query(instance,
"CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n", "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n")
settings={'show_table_uuid_in_table_create_query_if_not_nil': 1})
test_cluster.ddl_check_query(instance, "RENAME TABLE test_atomic.rmt TO test_atomic.rmt_renamed ON CLUSTER cluster") test_cluster.ddl_check_query(instance, "RENAME TABLE test_atomic.rmt TO test_atomic.rmt_renamed ON CLUSTER cluster")
test_cluster.ddl_check_query(instance, test_cluster.ddl_check_query(instance,
"CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n", "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n")
settings={'show_table_uuid_in_table_create_query_if_not_nil': 1})
test_cluster.ddl_check_query(instance, test_cluster.ddl_check_query(instance,
"EXCHANGE TABLES test_atomic.rmt AND test_atomic.rmt_renamed ON CLUSTER cluster") "EXCHANGE TABLES test_atomic.rmt AND test_atomic.rmt_renamed ON CLUSTER cluster")
test_cluster.pm_random_drops.push_rules(rules) test_cluster.pm_random_drops.push_rules(rules)

View File

@ -2,7 +2,6 @@
<profiles> <profiles>
<default> <default>
<optimize_trivial_count_query>0</optimize_trivial_count_query> <optimize_trivial_count_query>0</optimize_trivial_count_query>
<default_database_engine>Ordinary</default_database_engine>
</default> </default>
</profiles> </profiles>
</yandex> </yandex>

View File

@ -336,7 +336,7 @@ def test_polymorphic_parts_non_adaptive(start_cluster):
"Wide\t2\n") "Wide\t2\n")
assert node1.contains_in_log( assert node1.contains_in_log(
"<Warning> default.non_adaptive_table: Table can't create parts with adaptive granularity") "<Warning> default.non_adaptive_table ([0-9a-f-]*): Table can't create parts with adaptive granularity")
def test_in_memory(start_cluster): def test_in_memory(start_cluster):
@ -408,24 +408,29 @@ def test_in_memory_wal(start_cluster):
pm.partition_instances(node11, node12) pm.partition_instances(node11, node12)
check(node11, 300, 6) check(node11, 300, 6)
wal_file = os.path.join(node11.path, "database/data/default/wal_table/wal.bin") wal_file = "/var/lib/clickhouse/data/default/wal_table/wal.bin"
# Corrupt wal file # Corrupt wal file
open(wal_file, 'rw+').truncate(os.path.getsize(wal_file) - 10) # Truncate it to it's size minus 10 bytes
node11.exec_in_container(['bash', '-c', 'truncate --size="$(($(stat -c "%s" {}) - 10))" {}'.format(wal_file, wal_file)],
privileged=True, user='root')
node11.restart_clickhouse(kill=True) node11.restart_clickhouse(kill=True)
# Broken part is lost, but other restored successfully # Broken part is lost, but other restored successfully
check(node11, 250, 5) check(node11, 250, 5)
# WAL with blocks from 0 to 4 # WAL with blocks from 0 to 4
broken_wal_file = os.path.join(node11.path, "database/data/default/wal_table/wal_0_4.bin") broken_wal_file = "/var/lib/clickhouse/data/default/wal_table/wal_0_4.bin"
assert os.path.exists(broken_wal_file) # Check file exists
node11.exec_in_container(['bash', '-c', 'test -f {}'.format(broken_wal_file)])
# Fetch lost part from replica # Fetch lost part from replica
node11.query("SYSTEM SYNC REPLICA wal_table", timeout=20) node11.query("SYSTEM SYNC REPLICA wal_table", timeout=20)
check(node11, 300, 6) check(node11, 300, 6)
# Check that new data is written to new wal, but old is still exists for restoring # Check that new data is written to new wal, but old is still exists for restoring
assert os.path.getsize(wal_file) > 0 # Check file not empty
assert os.path.exists(broken_wal_file) node11.exec_in_container(['bash', '-c', 'test -s {}'.format(wal_file)])
# Check file exists
node11.exec_in_container(['bash', '-c', 'test -f {}'.format(broken_wal_file)])
# Data is lost without WAL # Data is lost without WAL
node11.query("ALTER TABLE wal_table MODIFY SETTING in_memory_parts_enable_wal = 0") node11.query("ALTER TABLE wal_table MODIFY SETTING in_memory_parts_enable_wal = 0")
@ -446,8 +451,8 @@ def test_in_memory_wal_rotate(start_cluster):
insert_random_data('restore_table', node11, 50) insert_random_data('restore_table', node11, 50)
for i in range(5): for i in range(5):
wal_file = os.path.join(node11.path, "database/data/default/restore_table/wal_{0}_{0}.bin".format(i)) # Check file exists
assert os.path.exists(wal_file) node11.exec_in_container(['bash', '-c', 'test -f /var/lib/clickhouse/data/default/restore_table/wal_{0}_{0}.bin'.format(i)])
for node in [node11, node12]: for node in [node11, node12]:
node.query( node.query(
@ -459,13 +464,14 @@ def test_in_memory_wal_rotate(start_cluster):
node11.restart_clickhouse(kill=True) node11.restart_clickhouse(kill=True)
for i in range(5): for i in range(5):
wal_file = os.path.join(node11.path, "database/data/default/restore_table/wal_{0}_{0}.bin".format(i)) # check file doesn't exist
assert not os.path.exists(wal_file) node11.exec_in_container(['bash', '-c', 'test ! -e /var/lib/clickhouse/data/default/restore_table/wal_{0}_{0}.bin'.format(i)])
# New wal file was created and ready to write part to it # New wal file was created and ready to write part to it
wal_file = os.path.join(node11.path, "database/data/default/restore_table/wal.bin") # Check file exists
assert os.path.exists(wal_file) node11.exec_in_container(['bash', '-c', 'test -f /var/lib/clickhouse/data/default/restore_table/wal.bin'])
assert os.path.getsize(wal_file) == 0 # Chech file empty
node11.exec_in_container(['bash', '-c', 'test ! -s /var/lib/clickhouse/data/default/restore_table/wal.bin'])
def test_in_memory_deduplication(start_cluster): def test_in_memory_deduplication(start_cluster):
@ -509,19 +515,20 @@ def test_in_memory_alters(start_cluster):
def test_polymorphic_parts_index(start_cluster): def test_polymorphic_parts_index(start_cluster):
node1.query('CREATE DATABASE test_index ENGINE=Ordinary')
node1.query(''' node1.query('''
CREATE TABLE index_compact(a UInt32, s String) CREATE TABLE test_index.index_compact(a UInt32, s String)
ENGINE = MergeTree ORDER BY a ENGINE = MergeTree ORDER BY a
SETTINGS min_rows_for_wide_part = 1000, index_granularity = 128, merge_max_block_size = 100''') SETTINGS min_rows_for_wide_part = 1000, index_granularity = 128, merge_max_block_size = 100''')
node1.query("INSERT INTO index_compact SELECT number, toString(number) FROM numbers(100)") node1.query("INSERT INTO test_index.index_compact SELECT number, toString(number) FROM numbers(100)")
node1.query("INSERT INTO index_compact SELECT number, toString(number) FROM numbers(30)") node1.query("INSERT INTO test_index.index_compact SELECT number, toString(number) FROM numbers(30)")
node1.query("OPTIMIZE TABLE index_compact FINAL") node1.query("OPTIMIZE TABLE test_index.index_compact FINAL")
assert node1.query("SELECT part_type FROM system.parts WHERE table = 'index_compact' AND active") == "Compact\n" assert node1.query("SELECT part_type FROM system.parts WHERE table = 'index_compact' AND active") == "Compact\n"
assert node1.query("SELECT marks FROM system.parts WHERE table = 'index_compact' AND active") == "2\n" assert node1.query("SELECT marks FROM system.parts WHERE table = 'index_compact' AND active") == "2\n"
index_path = os.path.join(node1.path, "database/data/default/index_compact/all_1_2_1/primary.idx") index_path = os.path.join(node1.path, "database/data/test_index/index_compact/all_1_2_1/primary.idx")
f = open(index_path, 'rb') f = open(index_path, 'rb')
assert os.path.getsize(index_path) == 8 assert os.path.getsize(index_path) == 8

View File

@ -0,0 +1,20 @@
<yandex>
<remote_servers>
<cluster>
<shard>
<replica>
<host>zero</host>
<port>9000</port>
</replica>
<replica>
<host>first</host>
<port>9000</port>
</replica>
<replica>
<host>second</host>
<port>9000</port>
</replica>
</shard>
</cluster>
</remote_servers>
</yandex>

View File

@ -7,23 +7,21 @@ from helpers.test_tools import TSV
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
zero = cluster.add_instance("zero", user_configs=["configs/users.d/settings.xml"], zero = cluster.add_instance("zero", user_configs=["configs/users.d/settings.xml"],
main_configs=["configs/config.d/remote_servers.xml"],
macros={"cluster": "anime", "shard": "0", "replica": "zero"}, macros={"cluster": "anime", "shard": "0", "replica": "zero"},
with_zookeeper=True) with_zookeeper=True)
first = cluster.add_instance("first", user_configs=["configs/users.d/settings.xml"], first = cluster.add_instance("first", user_configs=["configs/users.d/settings.xml"],
main_configs=["configs/config.d/remote_servers.xml"],
macros={"cluster": "anime", "shard": "0", "replica": "first"}, macros={"cluster": "anime", "shard": "0", "replica": "first"},
with_zookeeper=True) with_zookeeper=True)
second = cluster.add_instance("second", user_configs=["configs/users.d/settings.xml"], second = cluster.add_instance("second", user_configs=["configs/users.d/settings.xml"],
main_configs=["configs/config.d/remote_servers.xml"],
macros={"cluster": "anime", "shard": "0", "replica": "second"}, macros={"cluster": "anime", "shard": "0", "replica": "second"},
with_zookeeper=True) with_zookeeper=True)
def execute_on_all_cluster(query_):
for node in [zero, first, second]:
node.query(query_)
@pytest.fixture(scope="module") @pytest.fixture(scope="module")
def started_cluster(): def started_cluster():
global cluster global cluster
@ -36,7 +34,7 @@ def started_cluster():
def test_simple_add_replica(started_cluster): def test_simple_add_replica(started_cluster):
execute_on_all_cluster("DROP TABLE IF EXISTS test_simple") zero.query("DROP TABLE IF EXISTS test_simple ON CLUSTER cluster")
create_query = "CREATE TABLE test_simple " \ create_query = "CREATE TABLE test_simple " \
"(a Int8, d Date) " \ "(a Int8, d Date) " \
@ -67,11 +65,11 @@ def test_simple_add_replica(started_cluster):
assert '1\t2011-01-01\n' == first.query("SELECT * from test_simple") assert '1\t2011-01-01\n' == first.query("SELECT * from test_simple")
assert '1\t2011-01-01\n' == second.query("SELECT * from test_simple") assert '1\t2011-01-01\n' == second.query("SELECT * from test_simple")
execute_on_all_cluster("DROP TABLE IF EXISTS test_simple") zero.query("DROP TABLE IF EXISTS test_simple ON CLUSTER cluster")
def test_drop_replica_and_achieve_quorum(started_cluster): def test_drop_replica_and_achieve_quorum(started_cluster):
execute_on_all_cluster("DROP TABLE IF EXISTS test_drop_replica_and_achieve_quorum") zero.query("DROP TABLE IF EXISTS test_drop_replica_and_achieve_quorum ON CLUSTER cluster")
create_query = "CREATE TABLE test_drop_replica_and_achieve_quorum " \ create_query = "CREATE TABLE test_drop_replica_and_achieve_quorum " \
"(a Int8, d Date) " \ "(a Int8, d Date) " \
@ -125,7 +123,7 @@ def test_drop_replica_and_achieve_quorum(started_cluster):
assert TSV("1\t2011-01-01\n2\t2012-02-02\n") == TSV( assert TSV("1\t2011-01-01\n2\t2012-02-02\n") == TSV(
second.query("SELECT * FROM test_drop_replica_and_achieve_quorum ORDER BY a")) second.query("SELECT * FROM test_drop_replica_and_achieve_quorum ORDER BY a"))
execute_on_all_cluster("DROP TABLE IF EXISTS test_drop_replica_and_achieve_quorum") zero.query("DROP TABLE IF EXISTS test_drop_replica_and_achieve_quorum ON CLUSTER cluster")
@pytest.mark.parametrize( @pytest.mark.parametrize(
@ -136,17 +134,15 @@ def test_drop_replica_and_achieve_quorum(started_cluster):
] ]
) )
def test_insert_quorum_with_drop_partition(started_cluster, add_new_data): def test_insert_quorum_with_drop_partition(started_cluster, add_new_data):
execute_on_all_cluster("DROP TABLE IF EXISTS test_quorum_insert_with_drop_partition") zero.query("DROP TABLE IF EXISTS test_quorum_insert_with_drop_partition ON CLUSTER cluster")
create_query = "CREATE TABLE test_quorum_insert_with_drop_partition " \ create_query = "CREATE TABLE test_quorum_insert_with_drop_partition ON CLUSTER cluster " \
"(a Int8, d Date) " \ "(a Int8, d Date) " \
"Engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/{table}', '{replica}') " \ "Engine = ReplicatedMergeTree " \
"PARTITION BY d ORDER BY a " "PARTITION BY d ORDER BY a "
print("Create Replicated table with three replicas") print("Create Replicated table with three replicas")
zero.query(create_query) zero.query(create_query)
first.query(create_query)
second.query(create_query)
print("Stop fetches for test_quorum_insert_with_drop_partition at first replica.") print("Stop fetches for test_quorum_insert_with_drop_partition at first replica.")
first.query("SYSTEM STOP FETCHES test_quorum_insert_with_drop_partition") first.query("SYSTEM STOP FETCHES test_quorum_insert_with_drop_partition")
@ -167,9 +163,11 @@ def test_insert_quorum_with_drop_partition(started_cluster, add_new_data):
print("Sync first replica with others.") print("Sync first replica with others.")
first.query("SYSTEM SYNC REPLICA test_quorum_insert_with_drop_partition") first.query("SYSTEM SYNC REPLICA test_quorum_insert_with_drop_partition")
assert "20110101" not in first.query("SELECT * FROM system.zookeeper " \ assert "20110101" not in first.query("""
"where path='/clickhouse/tables/0/test_quorum_insert_with_drop_partition/quorum/last_part' " \ WITH (SELECT toString(uuid) FROM system.tables WHERE name = 'test_quorum_insert_with_drop_partition') AS uuid,
"format Vertical") '/clickhouse/tables/' || uuid || '/0/quorum/last_part' AS p
SELECT * FROM system.zookeeper WHERE path = p FORMAT Vertical
""")
print("Select from updated partition.") print("Select from updated partition.")
if (add_new_data): if (add_new_data):
@ -179,7 +177,7 @@ def test_insert_quorum_with_drop_partition(started_cluster, add_new_data):
assert TSV("") == TSV(zero.query("SELECT * FROM test_quorum_insert_with_drop_partition")) assert TSV("") == TSV(zero.query("SELECT * FROM test_quorum_insert_with_drop_partition"))
assert TSV("") == TSV(second.query("SELECT * FROM test_quorum_insert_with_drop_partition")) assert TSV("") == TSV(second.query("SELECT * FROM test_quorum_insert_with_drop_partition"))
execute_on_all_cluster("DROP TABLE IF EXISTS test_quorum_insert_with_drop_partition") zero.query("DROP TABLE IF EXISTS test_quorum_insert_with_drop_partition ON CLUSTER cluster")
@pytest.mark.parametrize( @pytest.mark.parametrize(
@ -190,28 +188,24 @@ def test_insert_quorum_with_drop_partition(started_cluster, add_new_data):
] ]
) )
def test_insert_quorum_with_move_partition(started_cluster, add_new_data): def test_insert_quorum_with_move_partition(started_cluster, add_new_data):
execute_on_all_cluster("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_source") zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_source ON CLUSTER cluster")
execute_on_all_cluster("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_destination") zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_destination ON CLUSTER cluster")
create_source = "CREATE TABLE test_insert_quorum_with_move_partition_source " \ create_source = "CREATE TABLE test_insert_quorum_with_move_partition_source ON CLUSTER cluster " \
"(a Int8, d Date) " \ "(a Int8, d Date) " \
"Engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/{table}', '{replica}') " \ "Engine = ReplicatedMergeTree " \
"PARTITION BY d ORDER BY a " "PARTITION BY d ORDER BY a "
create_destination = "CREATE TABLE test_insert_quorum_with_move_partition_destination " \ create_destination = "CREATE TABLE test_insert_quorum_with_move_partition_destination ON CLUSTER cluster " \
"(a Int8, d Date) " \ "(a Int8, d Date) " \
"Engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/{table}', '{replica}') " \ "Engine = ReplicatedMergeTree " \
"PARTITION BY d ORDER BY a " "PARTITION BY d ORDER BY a "
print("Create source Replicated table with three replicas") print("Create source Replicated table with three replicas")
zero.query(create_source) zero.query(create_source)
first.query(create_source)
second.query(create_source)
print("Create destination Replicated table with three replicas") print("Create destination Replicated table with three replicas")
zero.query(create_destination) zero.query(create_destination)
first.query(create_destination)
second.query(create_destination)
print("Stop fetches for test_insert_quorum_with_move_partition_source at first replica.") print("Stop fetches for test_insert_quorum_with_move_partition_source at first replica.")
first.query("SYSTEM STOP FETCHES test_insert_quorum_with_move_partition_source") first.query("SYSTEM STOP FETCHES test_insert_quorum_with_move_partition_source")
@ -233,9 +227,11 @@ def test_insert_quorum_with_move_partition(started_cluster, add_new_data):
print("Sync first replica with others.") print("Sync first replica with others.")
first.query("SYSTEM SYNC REPLICA test_insert_quorum_with_move_partition_source") first.query("SYSTEM SYNC REPLICA test_insert_quorum_with_move_partition_source")
assert "20110101" not in first.query("SELECT * FROM system.zookeeper " \ assert "20110101" not in first.query("""
"where path='/clickhouse/tables/0/test_insert_quorum_with_move_partition_source/quorum/last_part' " \ WITH (SELECT toString(uuid) FROM system.tables WHERE name = 'test_insert_quorum_with_move_partition_source') AS uuid,
"format Vertical") '/clickhouse/tables/' || uuid || '/0/quorum/last_part' AS p
SELECT * FROM system.zookeeper WHERE path = p FORMAT Vertical
""")
print("Select from updated partition.") print("Select from updated partition.")
if (add_new_data): if (add_new_data):
@ -246,12 +242,12 @@ def test_insert_quorum_with_move_partition(started_cluster, add_new_data):
assert TSV("") == TSV(zero.query("SELECT * FROM test_insert_quorum_with_move_partition_source")) assert TSV("") == TSV(zero.query("SELECT * FROM test_insert_quorum_with_move_partition_source"))
assert TSV("") == TSV(second.query("SELECT * FROM test_insert_quorum_with_move_partition_source")) assert TSV("") == TSV(second.query("SELECT * FROM test_insert_quorum_with_move_partition_source"))
execute_on_all_cluster("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_source") zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_source ON CLUSTER cluster")
execute_on_all_cluster("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_destination") zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_destination ON CLUSTER cluster")
def test_insert_quorum_with_ttl(started_cluster): def test_insert_quorum_with_ttl(started_cluster):
execute_on_all_cluster("DROP TABLE IF EXISTS test_insert_quorum_with_ttl") zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_ttl ON CLUSTER cluster")
create_query = "CREATE TABLE test_insert_quorum_with_ttl " \ create_query = "CREATE TABLE test_insert_quorum_with_ttl " \
"(a Int8, d Date) " \ "(a Int8, d Date) " \
@ -298,4 +294,4 @@ def test_insert_quorum_with_ttl(started_cluster):
assert TSV("2\t2012-02-02\n") == TSV( assert TSV("2\t2012-02-02\n") == TSV(
first.query("SELECT * FROM test_insert_quorum_with_ttl", settings={'select_sequential_consistency': 1})) first.query("SELECT * FROM test_insert_quorum_with_ttl", settings={'select_sequential_consistency': 1}))
execute_on_all_cluster("DROP TABLE IF EXISTS test_insert_quorum_with_ttl") zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_ttl ON CLUSTER cluster")

View File

@ -22,4 +22,28 @@
<merge_tree> <merge_tree>
<min_bytes_for_wide_part>0</min_bytes_for_wide_part> <min_bytes_for_wide_part>0</min_bytes_for_wide_part>
</merge_tree> </merge_tree>
<remote_servers>
<cluster>
<shard>
<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>
</cluster>
</remote_servers>
<macros>
<shard>0</shard>
</macros>
</yandex> </yandex>

View File

@ -14,11 +14,11 @@ def cluster():
try: try:
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
cluster.add_instance("node1", main_configs=["configs/config.d/storage_conf.xml"], macros={'cluster': 'test1'}, cluster.add_instance("node1", main_configs=["configs/config.d/storage_conf.xml"], macros={'replica': '1'},
with_minio=True, with_zookeeper=True) with_minio=True, with_zookeeper=True)
cluster.add_instance("node2", main_configs=["configs/config.d/storage_conf.xml"], macros={'cluster': 'test1'}, cluster.add_instance("node2", main_configs=["configs/config.d/storage_conf.xml"], macros={'replica': '2'},
with_zookeeper=True) with_zookeeper=True)
cluster.add_instance("node3", main_configs=["configs/config.d/storage_conf.xml"], macros={'cluster': 'test1'}, cluster.add_instance("node3", main_configs=["configs/config.d/storage_conf.xml"], macros={'replica': '3'},
with_zookeeper=True) with_zookeeper=True)
logging.info("Starting cluster...") logging.info("Starting cluster...")
@ -49,12 +49,12 @@ def generate_values(date_str, count, sign=1):
def create_table(cluster, additional_settings=None): def create_table(cluster, additional_settings=None):
create_table_statement = """ create_table_statement = """
CREATE TABLE s3_test ( CREATE TABLE s3_test ON CLUSTER cluster(
dt Date, dt Date,
id Int64, id Int64,
data String, data String,
INDEX min_max (id) TYPE minmax GRANULARITY 3 INDEX min_max (id) TYPE minmax GRANULARITY 3
) ENGINE=ReplicatedMergeTree('/clickhouse/{cluster}/tables/test/s3', '{instance}') ) ENGINE=ReplicatedMergeTree()
PARTITION BY dt PARTITION BY dt
ORDER BY (dt, id) ORDER BY (dt, id)
SETTINGS storage_policy='s3' SETTINGS storage_policy='s3'
@ -63,8 +63,7 @@ def create_table(cluster, additional_settings=None):
create_table_statement += "," create_table_statement += ","
create_table_statement += additional_settings create_table_statement += additional_settings
for node in cluster.instances.values(): cluster.instances.values()[0].query(create_table_statement)
node.query(create_table_statement)
@pytest.fixture(autouse=True) @pytest.fixture(autouse=True)

View File

@ -34,7 +34,7 @@ def started_cluster():
for current_node in nodes: for current_node in nodes:
current_node.query(''' current_node.query('''
CREATE DATABASE mydb ENGINE=Ordinary; CREATE DATABASE mydb;
CREATE TABLE mydb.filtered_table1 (a UInt8, b UInt8) ENGINE MergeTree ORDER BY a; CREATE TABLE mydb.filtered_table1 (a UInt8, b UInt8) ENGINE MergeTree ORDER BY a;
INSERT INTO mydb.filtered_table1 values (0, 0), (0, 1), (1, 0), (1, 1); INSERT INTO mydb.filtered_table1 values (0, 0), (0, 1), (1, 0), (1, 1);
@ -360,7 +360,7 @@ def test_miscellaneous_engines():
# ReplicatedCollapsingMergeTree # ReplicatedCollapsingMergeTree
node.query("DROP TABLE mydb.filtered_table1") node.query("DROP TABLE mydb.filtered_table1")
node.query( node.query(
"CREATE TABLE mydb.filtered_table1 (a UInt8, b Int8) ENGINE ReplicatedCollapsingMergeTree('/clickhouse/tables/00-00/filtered_table1', 'replica1', b) ORDER BY a") "CREATE TABLE mydb.filtered_table1 (a UInt8, b Int8) ENGINE ReplicatedCollapsingMergeTree('/clickhouse/tables/00-01/filtered_table1', 'replica1', b) ORDER BY a")
node.query("INSERT INTO mydb.filtered_table1 values (0, 1), (0, 1), (1, 1), (1, 1)") node.query("INSERT INTO mydb.filtered_table1 values (0, 1), (0, 1), (1, 1), (1, 1)")
assert node.query("SELECT * FROM mydb.filtered_table1") == TSV([[1, 1], [1, 1]]) assert node.query("SELECT * FROM mydb.filtered_table1") == TSV([[1, 1], [1, 1]])

View File

@ -3,13 +3,13 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh . "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS database_for_dict" $CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS db_01038"
$CLICKHOUSE_CLIENT --query "CREATE DATABASE database_for_dict Engine = Ordinary" $CLICKHOUSE_CLIENT --query "CREATE DATABASE db_01038"
$CLICKHOUSE_CLIENT --query " $CLICKHOUSE_CLIENT --query "
CREATE TABLE database_for_dict.table_for_dict CREATE TABLE db_01038.table_for_dict
( (
key_column UInt64, key_column UInt64,
value Float64 value Float64
@ -17,34 +17,34 @@ CREATE TABLE database_for_dict.table_for_dict
ENGINE = MergeTree() ENGINE = MergeTree()
ORDER BY key_column" ORDER BY key_column"
$CLICKHOUSE_CLIENT --query "INSERT INTO database_for_dict.table_for_dict VALUES (1, 1.1)" $CLICKHOUSE_CLIENT --query "INSERT INTO db_01038.table_for_dict VALUES (1, 1.1)"
$CLICKHOUSE_CLIENT --query " $CLICKHOUSE_CLIENT --query "
CREATE DICTIONARY database_for_dict.dict_with_zero_min_lifetime CREATE DICTIONARY db_01038.dict_with_zero_min_lifetime
( (
key_column UInt64, key_column UInt64,
value Float64 DEFAULT 77.77 value Float64 DEFAULT 77.77
) )
PRIMARY KEY key_column PRIMARY KEY key_column
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' DB 'database_for_dict')) SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' DB 'db_01038'))
LIFETIME(1) LIFETIME(1)
LAYOUT(FLAT())" LAYOUT(FLAT())"
$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('database_for_dict.dict_with_zero_min_lifetime', 'value', toUInt64(1))" $CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('db_01038.dict_with_zero_min_lifetime', 'value', toUInt64(1))"
$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('database_for_dict.dict_with_zero_min_lifetime', 'value', toUInt64(2))" $CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('db_01038.dict_with_zero_min_lifetime', 'value', toUInt64(2))"
$CLICKHOUSE_CLIENT --query "INSERT INTO database_for_dict.table_for_dict VALUES (2, 2.2)" $CLICKHOUSE_CLIENT --query "INSERT INTO db_01038.table_for_dict VALUES (2, 2.2)"
function check() function check()
{ {
query_result=$($CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('database_for_dict.dict_with_zero_min_lifetime', 'value', toUInt64(2))") query_result=$($CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('db_01038.dict_with_zero_min_lifetime', 'value', toUInt64(2))")
while [ "$query_result" != "2.2" ] while [ "$query_result" != "2.2" ]
do do
query_result=$($CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('database_for_dict.dict_with_zero_min_lifetime', 'value', toUInt64(2))") query_result=$($CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('db_01038.dict_with_zero_min_lifetime', 'value', toUInt64(2))")
done done
} }
@ -53,8 +53,8 @@ export -f check;
timeout 10 bash -c check timeout 10 bash -c check
$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('database_for_dict.dict_with_zero_min_lifetime', 'value', toUInt64(1))" $CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('db_01038.dict_with_zero_min_lifetime', 'value', toUInt64(1))"
$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('database_for_dict.dict_with_zero_min_lifetime', 'value', toUInt64(2))" $CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('db_01038.dict_with_zero_min_lifetime', 'value', toUInt64(2))"
$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS database_for_dict" $CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS db_01038"

View File

@ -1,12 +1,12 @@
DROP DATABASE IF EXISTS database_for_dict; DROP DATABASE IF EXISTS database_for_dict_01268;
CREATE DATABASE database_for_dict Engine = Ordinary; CREATE DATABASE database_for_dict_01268;
DROP TABLE IF EXISTS database_for_dict.table_for_dict1; DROP TABLE IF EXISTS database_for_dict_01268.table_for_dict1;
DROP TABLE IF EXISTS database_for_dict.table_for_dict2; DROP TABLE IF EXISTS database_for_dict_01268.table_for_dict2;
DROP TABLE IF EXISTS database_for_dict.table_for_dict3; DROP TABLE IF EXISTS database_for_dict_01268.table_for_dict3;
CREATE TABLE database_for_dict.table_for_dict1 CREATE TABLE database_for_dict_01268.table_for_dict1
( (
key_column UInt64, key_column UInt64,
second_column UInt64, second_column UInt64,
@ -15,9 +15,9 @@ CREATE TABLE database_for_dict.table_for_dict1
ENGINE = MergeTree() ENGINE = MergeTree()
ORDER BY key_column; ORDER BY key_column;
INSERT INTO database_for_dict.table_for_dict1 VALUES (100500, 10000000, 'Hello world'); INSERT INTO database_for_dict_01268.table_for_dict1 VALUES (100500, 10000000, 'Hello world');
CREATE TABLE database_for_dict.table_for_dict2 CREATE TABLE database_for_dict_01268.table_for_dict2
( (
region_id UInt64, region_id UInt64,
parent_region UInt64, parent_region UInt64,
@ -26,13 +26,13 @@ CREATE TABLE database_for_dict.table_for_dict2
ENGINE = MergeTree() ENGINE = MergeTree()
ORDER BY region_id; ORDER BY region_id;
INSERT INTO database_for_dict.table_for_dict2 VALUES (1, 0, 'Russia'); INSERT INTO database_for_dict_01268.table_for_dict2 VALUES (1, 0, 'Russia');
INSERT INTO database_for_dict.table_for_dict2 VALUES (2, 1, 'Moscow'); INSERT INTO database_for_dict_01268.table_for_dict2 VALUES (2, 1, 'Moscow');
INSERT INTO database_for_dict.table_for_dict2 VALUES (3, 2, 'Center'); INSERT INTO database_for_dict_01268.table_for_dict2 VALUES (3, 2, 'Center');
INSERT INTO database_for_dict.table_for_dict2 VALUES (4, 0, 'Great Britain'); INSERT INTO database_for_dict_01268.table_for_dict2 VALUES (4, 0, 'Great Britain');
INSERT INTO database_for_dict.table_for_dict2 VALUES (5, 4, 'London'); INSERT INTO database_for_dict_01268.table_for_dict2 VALUES (5, 4, 'London');
CREATE TABLE database_for_dict.table_for_dict3 CREATE TABLE database_for_dict_01268.table_for_dict3
( (
region_id UInt64, region_id UInt64,
parent_region Float32, parent_region Float32,
@ -41,91 +41,91 @@ CREATE TABLE database_for_dict.table_for_dict3
ENGINE = MergeTree() ENGINE = MergeTree()
ORDER BY region_id; ORDER BY region_id;
INSERT INTO database_for_dict.table_for_dict3 VALUES (1, 0.5, 'Russia'); INSERT INTO database_for_dict_01268.table_for_dict3 VALUES (1, 0.5, 'Russia');
INSERT INTO database_for_dict.table_for_dict3 VALUES (2, 1.6, 'Moscow'); INSERT INTO database_for_dict_01268.table_for_dict3 VALUES (2, 1.6, 'Moscow');
INSERT INTO database_for_dict.table_for_dict3 VALUES (3, 2.3, 'Center'); INSERT INTO database_for_dict_01268.table_for_dict3 VALUES (3, 2.3, 'Center');
INSERT INTO database_for_dict.table_for_dict3 VALUES (4, 0.2, 'Great Britain'); INSERT INTO database_for_dict_01268.table_for_dict3 VALUES (4, 0.2, 'Great Britain');
INSERT INTO database_for_dict.table_for_dict3 VALUES (5, 4.9, 'London'); INSERT INTO database_for_dict_01268.table_for_dict3 VALUES (5, 4.9, 'London');
DROP DATABASE IF EXISTS ordinary_db; DROP DATABASE IF EXISTS db_01268;
CREATE DATABASE ordinary_db ENGINE = Ordinary; CREATE DATABASE db_01268;
DROP DICTIONARY IF EXISTS ordinary_db.dict1; DROP DICTIONARY IF EXISTS db_01268.dict1;
DROP DICTIONARY IF EXISTS ordinary_db.dict2; DROP DICTIONARY IF EXISTS db_01268.dict2;
DROP DICTIONARY IF EXISTS ordinary_db.dict3; DROP DICTIONARY IF EXISTS db_01268.dict3;
CREATE DICTIONARY ordinary_db.dict1 CREATE DICTIONARY db_01268.dict1
( (
key_column UInt64 DEFAULT 0, key_column UInt64 DEFAULT 0,
second_column UInt64 DEFAULT 1, second_column UInt64 DEFAULT 1,
third_column String DEFAULT 'qqq' third_column String DEFAULT 'qqq'
) )
PRIMARY KEY key_column PRIMARY KEY key_column
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' PASSWORD '' DB 'database_for_dict')) SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' PASSWORD '' DB 'database_for_dict_01268'))
LAYOUT(DIRECT()) SETTINGS(max_result_bytes=1); LAYOUT(DIRECT()) SETTINGS(max_result_bytes=1);
CREATE DICTIONARY ordinary_db.dict2 CREATE DICTIONARY db_01268.dict2
( (
region_id UInt64 DEFAULT 0, region_id UInt64 DEFAULT 0,
parent_region UInt64 DEFAULT 0 HIERARCHICAL, parent_region UInt64 DEFAULT 0 HIERARCHICAL,
region_name String DEFAULT '' region_name String DEFAULT ''
) )
PRIMARY KEY region_id PRIMARY KEY region_id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict2' PASSWORD '' DB 'database_for_dict')) SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict2' PASSWORD '' DB 'database_for_dict_01268'))
LAYOUT(DIRECT()); LAYOUT(DIRECT());
CREATE DICTIONARY ordinary_db.dict3 CREATE DICTIONARY db_01268.dict3
( (
region_id UInt64 DEFAULT 0, region_id UInt64 DEFAULT 0,
parent_region Float32 DEFAULT 0, parent_region Float32 DEFAULT 0,
region_name String DEFAULT '' region_name String DEFAULT ''
) )
PRIMARY KEY region_id PRIMARY KEY region_id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict3' PASSWORD '' DB 'database_for_dict')) SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict3' PASSWORD '' DB 'database_for_dict_01268'))
LAYOUT(DIRECT()); LAYOUT(DIRECT());
SELECT 'INITIALIZING DICTIONARY'; SELECT 'INITIALIZING DICTIONARY';
SELECT dictGetHierarchy('ordinary_db.dict2', toUInt64(3)); SELECT dictGetHierarchy('db_01268.dict2', toUInt64(3));
SELECT dictHas('ordinary_db.dict2', toUInt64(3)); SELECT dictHas('db_01268.dict2', toUInt64(3));
SELECT dictHas('ordinary_db.dict2', toUInt64(45)); SELECT dictHas('db_01268.dict2', toUInt64(45));
SELECT dictIsIn('ordinary_db.dict2', toUInt64(3), toUInt64(1)); SELECT dictIsIn('db_01268.dict2', toUInt64(3), toUInt64(1));
SELECT dictIsIn('ordinary_db.dict2', toUInt64(1), toUInt64(3)); SELECT dictIsIn('db_01268.dict2', toUInt64(1), toUInt64(3));
SELECT dictGetUInt64('ordinary_db.dict2', 'parent_region', toUInt64(3)); SELECT dictGetUInt64('db_01268.dict2', 'parent_region', toUInt64(3));
SELECT dictGetUInt64('ordinary_db.dict2', 'parent_region', toUInt64(99)); SELECT dictGetUInt64('db_01268.dict2', 'parent_region', toUInt64(99));
SELECT dictGetFloat32('ordinary_db.dict3', 'parent_region', toUInt64(3)); SELECT dictGetFloat32('db_01268.dict3', 'parent_region', toUInt64(3));
SELECT dictGetFloat32('ordinary_db.dict3', 'parent_region', toUInt64(2)); SELECT dictGetFloat32('db_01268.dict3', 'parent_region', toUInt64(2));
SELECT dictGetFloat32('ordinary_db.dict3', 'parent_region', toUInt64(1)); SELECT dictGetFloat32('db_01268.dict3', 'parent_region', toUInt64(1));
SELECT dictGetString('ordinary_db.dict2', 'region_name', toUInt64(5)); SELECT dictGetString('db_01268.dict2', 'region_name', toUInt64(5));
SELECT dictGetString('ordinary_db.dict2', 'region_name', toUInt64(4)); SELECT dictGetString('db_01268.dict2', 'region_name', toUInt64(4));
SELECT dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(100), 'NONE'); SELECT dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(100), 'NONE');
SELECT number + 1, dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(number + 1), 'NONE') chars FROM numbers(10); SELECT number + 1, dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(number + 1), 'NONE') chars FROM numbers(10);
SELECT number + 1, dictGetFloat32OrDefault('ordinary_db.dict3', 'parent_region', toUInt64(number + 1), toFloat32(0)) chars FROM numbers(10); SELECT number + 1, dictGetFloat32OrDefault('db_01268.dict3', 'parent_region', toUInt64(number + 1), toFloat32(0)) chars FROM numbers(10);
SELECT dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(1), 'NONE'); SELECT dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(1), 'NONE');
SELECT dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(2), 'NONE'); SELECT dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(2), 'NONE');
SELECT dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(3), 'NONE'); SELECT dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(3), 'NONE');
SELECT dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(4), 'NONE'); SELECT dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(4), 'NONE');
SELECT dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(5), 'NONE'); SELECT dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(5), 'NONE');
SELECT dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(6), 'NONE'); SELECT dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(6), 'NONE');
SELECT dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(7), 'NONE'); SELECT dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(7), 'NONE');
SELECT dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(8), 'NONE'); SELECT dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(8), 'NONE');
SELECT dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(9), 'NONE'); SELECT dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(9), 'NONE');
SELECT dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(10), 'NONE'); SELECT dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(10), 'NONE');
SELECT dictGetUInt64('ordinary_db.dict1', 'second_column', toUInt64(100500)); -- { serverError 396 } SELECT dictGetUInt64('db_01268.dict1', 'second_column', toUInt64(100500)); -- { serverError 396 }
SELECT 'END'; SELECT 'END';
DROP DICTIONARY IF EXISTS ordinary_db.dict1; DROP DICTIONARY IF EXISTS db_01268.dict1;
DROP DICTIONARY IF EXISTS ordinary_db.dict2; DROP DICTIONARY IF EXISTS db_01268.dict2;
DROP DICTIONARY IF EXISTS ordinary_db.dict3; DROP DICTIONARY IF EXISTS db_01268.dict3;
DROP DATABASE IF EXISTS ordinary_db; DROP DATABASE IF EXISTS db_01268;
DROP TABLE IF EXISTS database_for_dict.table_for_dict1; DROP TABLE IF EXISTS database_for_dict_01268.table_for_dict1;
DROP TABLE IF EXISTS database_for_dict.table_for_dict2; DROP TABLE IF EXISTS database_for_dict_01268.table_for_dict2;
DROP TABLE IF EXISTS database_for_dict.table_for_dict3; DROP TABLE IF EXISTS database_for_dict_01268.table_for_dict3;
DROP DATABASE IF EXISTS database_for_dict; DROP DATABASE IF EXISTS database_for_dict_01268;

View File

@ -89,18 +89,5 @@
"release-build": [ "release-build": [
], ],
"database-atomic": [ "database-atomic": [
/// Inner tables of materialized views have different names
///"00738_lock_for_inner_table",
///"00609_mv_index_in_in",
///"00510_materizlized_view_and_deduplication_zookeeper",
/// Different database engine
///"00604_show_create_database",
/// UUID must be specified in ATTACH TABLE
///"01190_full_attach_syntax",
/// Assumes blocking DROP
///"01320_create_sync_race_condition",
/// Internal distionary name is different
///"01225_show_create_table_from_dictionary",
///"01224_no_superfluous_dict_reload"
] ]
} }