enable more tests

This commit is contained in:
Alexander Tokmakov 2020-09-22 14:56:40 +03:00
parent dc1cc11764
commit 7fcf20e48c
44 changed files with 130 additions and 107 deletions

View File

@ -525,7 +525,7 @@ static bool allowDictJoin(StoragePtr joined_storage, const Context & context, St
if (!dict) if (!dict)
return false; return false;
dict_name = dict->dictionaryName(); dict_name = dict->resolvedDictionaryName();
auto dictionary = context.getExternalDictionariesLoader().getDictionary(dict_name); auto dictionary = context.getExternalDictionariesLoader().getDictionary(dict_name);
if (!dictionary) if (!dictionary)
return false; return false;

View File

@ -92,6 +92,12 @@ String StorageDictionary::generateNamesAndTypesDescription(const NamesAndTypesLi
return ss.str(); return ss.str();
} }
String StorageDictionary::resolvedDictionaryName() const
{
if (location == Location::SameDatabaseAndNameAsDictionary)
return dictionary_name;
return DatabaseCatalog::instance().resolveDictionaryName(dictionary_name);
}
StorageDictionary::StorageDictionary( StorageDictionary::StorageDictionary(
const StorageID & table_id_, const StorageID & table_id_,
@ -100,9 +106,6 @@ StorageDictionary::StorageDictionary(
Location location_) Location location_)
: IStorage(table_id_) : IStorage(table_id_)
, dictionary_name(dictionary_name_) , dictionary_name(dictionary_name_)
, resolved_dictionary_name(location_ == Location::SameDatabaseAndNameAsDictionary
? dictionary_name
: DatabaseCatalog::instance().resolveDictionaryName(dictionary_name))
, location(location_) , location(location_)
{ {
StorageInMemoryMetadata storage_metadata; StorageInMemoryMetadata storage_metadata;
@ -135,7 +138,7 @@ Pipe StorageDictionary::read(
const size_t max_block_size, const size_t max_block_size,
const unsigned /*threads*/) const unsigned /*threads*/)
{ {
auto dictionary = context.getExternalDictionariesLoader().getDictionary(resolved_dictionary_name); auto dictionary = context.getExternalDictionariesLoader().getDictionary(resolvedDictionaryName());
auto stream = dictionary->getBlockInputStream(column_names, max_block_size); auto stream = dictionary->getBlockInputStream(column_names, max_block_size);
/// TODO: update dictionary interface for processors. /// TODO: update dictionary interface for processors.
return Pipe(std::make_shared<SourceFromInputStream>(stream)); return Pipe(std::make_shared<SourceFromInputStream>(stream));

View File

@ -28,7 +28,8 @@ public:
static NamesAndTypesList getNamesAndTypes(const DictionaryStructure & dictionary_structure); static NamesAndTypesList getNamesAndTypes(const DictionaryStructure & dictionary_structure);
static String generateNamesAndTypesDescription(const NamesAndTypesList & list); static String generateNamesAndTypesDescription(const NamesAndTypesList & list);
const String & dictionaryName() const { return resolved_dictionary_name; } const String & dictionaryName() const { return dictionary_name; }
String resolvedDictionaryName() const;
/// Specifies where the table is located relative to the dictionary. /// Specifies where the table is located relative to the dictionary.
enum class Location enum class Location
@ -50,7 +51,6 @@ public:
private: private:
const String dictionary_name; const String dictionary_name;
const String resolved_dictionary_name;
const Location location; const Location location;
protected: protected:

View File

@ -13,7 +13,7 @@ node1 = cluster.add_instance('node1', main_configs=["configs/config.d/zookeeper_
def start_cluster(): def start_cluster():
try: try:
cluster.start() cluster.start()
node1.query("CREATE DATABASE zktest ENGINE=Ordinary;") node1.query("CREATE DATABASE zktest ENGINE=Ordinary;") # Different behaviour with Atomic
node1.query( node1.query(
''' '''
CREATE TABLE zktest.atomic_drop_table (n UInt32) CREATE TABLE zktest.atomic_drop_table (n UInt32)

View File

@ -14,7 +14,7 @@ path_to_data = '/var/lib/clickhouse/'
def started_cluster(): def started_cluster():
try: try:
cluster.start() cluster.start()
q('CREATE DATABASE test ENGINE = Ordinary') q('CREATE DATABASE test ENGINE = Ordinary') # Different path in shadow/ with Atomic
yield cluster yield cluster

View File

@ -142,7 +142,7 @@ 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 DATABASE test ENGINE=Ordinary") node4.query("CREATE DATABASE test ENGINE=Ordinary") # Different path in shadow/ with Atomic
node4.query("CREATE TABLE test.backup_table(A Int64, B String, C Date) Engine = MergeTree order by tuple()") node4.query("CREATE TABLE test.backup_table(A Int64, B String, C Date) Engine = MergeTree order by tuple()")

View File

@ -33,7 +33,7 @@
<enabled_partitions>3 4 5 6 1 2 0 </enabled_partitions> <enabled_partitions>3 4 5 6 1 2 0 </enabled_partitions>
<!-- Engine of destination tables --> <!-- Engine of destination tables -->
<engine>ENGINE=ReplicatedMergeTree PARTITION BY d % 3 ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d) SETTINGS index_granularity = 16</engine> <engine>ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster{cluster}/{shard}/hits', '{replica}') PARTITION BY d % 3 ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d) SETTINGS index_granularity = 16</engine>
<!-- Which sarding key to use while copying --> <!-- Which sarding key to use while copying -->
<sharding_key>d + 1</sharding_key> <sharding_key>d + 1</sharding_key>
@ -93,4 +93,4 @@
</cluster1> </cluster1>
</remote_servers> </remote_servers>
</yandex> </yandex>

View File

@ -34,7 +34,7 @@
<!-- Engine of destination tables --> <!-- Engine of destination tables -->
<engine>ENGINE= <engine>ENGINE=
ReplicatedMergeTree ReplicatedMergeTree('/clickhouse/tables/cluster{cluster}/{shard}/b', '{replica}')
PARTITION BY toMonday(date) PARTITION BY toMonday(date)
ORDER BY d ORDER BY d
</engine> </engine>
@ -97,4 +97,4 @@
</cluster1> </cluster1>
</remote_servers> </remote_servers>
</yandex> </yandex>

View File

@ -28,7 +28,7 @@
<!-- Engine of destination tables --> <!-- Engine of destination tables -->
<engine>ENGINE= <engine>ENGINE=
ReplicatedMergeTree ReplicatedMergeTree('/clickhouse/tables/cluster{cluster}/{shard}/test_block_size', '{replica}')
ORDER BY d PARTITION BY partition ORDER BY d PARTITION BY partition
</engine> </engine>
@ -99,4 +99,4 @@
</shard_0_0> </shard_0_0>
</remote_servers> </remote_servers>
</yandex> </yandex>

View File

@ -81,11 +81,11 @@ class Task1:
for cluster_num in ["0", "1"]: for cluster_num in ["0", "1"]:
ddl_check_query(instance, "DROP DATABASE IF EXISTS default ON CLUSTER cluster{}".format(cluster_num)) ddl_check_query(instance, "DROP DATABASE IF EXISTS default ON CLUSTER cluster{}".format(cluster_num))
ddl_check_query(instance, ddl_check_query(instance,
"CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{}".format( "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{} ENGINE=Ordinary".format(
cluster_num)) cluster_num))
ddl_check_query(instance, "CREATE TABLE hits ON CLUSTER cluster0 (d UInt64, d1 UInt64 MATERIALIZED d+1) " + ddl_check_query(instance, "CREATE TABLE hits ON CLUSTER cluster0 (d UInt64, d1 UInt64 MATERIALIZED d+1) " +
"ENGINE=ReplicatedMergeTree " + "ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/hits', '{replica}') " +
"PARTITION BY d % 3 ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d) SETTINGS index_granularity = 16") "PARTITION BY d % 3 ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d) SETTINGS index_granularity = 16")
ddl_check_query(instance, ddl_check_query(instance,
"CREATE TABLE hits_all ON CLUSTER cluster0 (d UInt64) ENGINE=Distributed(cluster0, default, hits, d)") "CREATE TABLE hits_all ON CLUSTER cluster0 (d UInt64) ENGINE=Distributed(cluster0, default, hits, d)")
@ -110,11 +110,10 @@ class Task1:
class Task2: class Task2:
def __init__(self, cluster, unique_zk_path): def __init__(self, cluster):
self.cluster = cluster self.cluster = cluster
self.zk_task_path = "/clickhouse-copier/task_month_to_week_partition" self.zk_task_path = "/clickhouse-copier/task_month_to_week_partition"
self.copier_task_config = open(os.path.join(CURRENT_TEST_DIR, 'task_month_to_week_description.xml'), 'r').read() self.copier_task_config = open(os.path.join(CURRENT_TEST_DIR, 'task_month_to_week_description.xml'), 'r').read()
self.unique_zk_path = unique_zk_path
def start(self): def start(self):
instance = cluster.instances['s0_0_0'] instance = cluster.instances['s0_0_0']
@ -122,13 +121,11 @@ class Task2:
for cluster_num in ["0", "1"]: for cluster_num in ["0", "1"]:
ddl_check_query(instance, "DROP DATABASE IF EXISTS default ON CLUSTER cluster{}".format(cluster_num)) ddl_check_query(instance, "DROP DATABASE IF EXISTS default ON CLUSTER cluster{}".format(cluster_num))
ddl_check_query(instance, ddl_check_query(instance,
"CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{}".format( "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{} ENGINE=Ordinary".format(
cluster_num)) cluster_num))
ddl_check_query(instance, ddl_check_query(instance,
"CREATE TABLE a ON CLUSTER cluster0 (date Date, d UInt64, d1 UInt64 ALIAS d+1) " "CREATE TABLE a ON CLUSTER cluster0 (date Date, d UInt64, d1 UInt64 ALIAS d+1) ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/a', '{replica}', date, intHash64(d), (date, intHash64(d)), 8192)")
"ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/" + self.unique_zk_path + "', "
"'{replica}', date, intHash64(d), (date, intHash64(d)), 8192)")
ddl_check_query(instance, ddl_check_query(instance,
"CREATE TABLE a_all ON CLUSTER cluster0 (date Date, d UInt64) ENGINE=Distributed(cluster0, default, a, d)") "CREATE TABLE a_all ON CLUSTER cluster0 (date Date, d UInt64) ENGINE=Distributed(cluster0, default, a, d)")
@ -172,7 +169,7 @@ class Task_test_block_size:
ddl_check_query(instance, """ ddl_check_query(instance, """
CREATE TABLE test_block_size ON CLUSTER shard_0_0 (partition Date, d UInt64) CREATE TABLE test_block_size ON CLUSTER shard_0_0 (partition Date, d UInt64)
ENGINE=ReplicatedMergeTree ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/test_block_size', '{replica}')
ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d)""", 2) ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d)""", 2)
instance.query( instance.query(
@ -335,17 +332,17 @@ def test_copy_with_recovering_after_move_faults(started_cluster, use_sample_offs
@pytest.mark.timeout(600) @pytest.mark.timeout(600)
def test_copy_month_to_week_partition(started_cluster): def test_copy_month_to_week_partition(started_cluster):
execute_task(Task2(started_cluster, "test1"), []) execute_task(Task2(started_cluster), [])
@pytest.mark.timeout(600) @pytest.mark.timeout(600)
def test_copy_month_to_week_partition_with_recovering(started_cluster): def test_copy_month_to_week_partition_with_recovering(started_cluster):
execute_task(Task2(started_cluster, "test2"), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) execute_task(Task2(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)])
@pytest.mark.timeout(600) @pytest.mark.timeout(600)
def test_copy_month_to_week_partition_with_recovering_after_move_faults(started_cluster): def test_copy_month_to_week_partition_with_recovering_after_move_faults(started_cluster):
execute_task(Task2(started_cluster, "test3"), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) execute_task(Task2(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)])
def test_block_size(started_cluster): def test_block_size(started_cluster):

View File

@ -59,7 +59,7 @@ class TaskTrivial:
for node in [source, destination]: for node in [source, destination]:
node.query("DROP DATABASE IF EXISTS default") node.query("DROP DATABASE IF EXISTS default")
node.query("CREATE DATABASE IF NOT EXISTS default") node.query("CREATE DATABASE IF NOT EXISTS default ENGINE=Ordinary")
source.query("CREATE TABLE trivial (d UInt64, d1 UInt64 MATERIALIZED d+1) " source.query("CREATE TABLE trivial (d UInt64, d1 UInt64 MATERIALIZED d+1) "
"ENGINE=ReplicatedMergeTree('/clickhouse/tables/source_trivial_cluster/1/trivial', '1') " "ENGINE=ReplicatedMergeTree('/clickhouse/tables/source_trivial_cluster/1/trivial', '1') "

View File

@ -12,15 +12,18 @@ def start_cluster():
try: try:
cluster.start() cluster.start()
for node in nodes: for node in nodes:
node.query("CREATE DATABASE IF NOT EXISTS test ENGINE=Ordinary") 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 atest") node.query("CREATE DATABASE IF NOT EXISTS atest")
node.query("CREATE DATABASE IF NOT EXISTS ztest") node.query("CREATE DATABASE IF NOT EXISTS ztest")
node.query("CREATE TABLE test.source(x UInt64, y UInt64) ENGINE=Log") node.query("CREATE TABLE test.source(x UInt64, y UInt64) ENGINE=Log")
node.query("INSERT INTO test.source VALUES (5,6)") node.query("INSERT INTO test.source VALUES (5,6)")
node.query("CREATE DICTIONARY test.dict(x UInt64, y UInt64) PRIMARY KEY x " \ for db in ("test", "test_ordinary"):
"SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'source' DB 'test')) " \ node.query("CREATE DICTIONARY {}.dict(x UInt64, y UInt64) PRIMARY KEY x " \
"LAYOUT(FLAT()) LIFETIME(0)") "SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'source' DB 'test')) " \
"LAYOUT(FLAT()) LIFETIME(0)".format(db))
yield cluster yield cluster
finally: finally:
@ -91,10 +94,10 @@ def test_dependency_via_explicit_table(node):
def test_dependency_via_dictionary_database(node): def test_dependency_via_dictionary_database(node):
node.query("CREATE DATABASE dict_db ENGINE=Dictionary") node.query("CREATE DATABASE dict_db ENGINE=Dictionary")
d_names = ["test.adict", "test.zdict", "atest.dict", "ztest.dict"] d_names = ["test_ordinary.adict", "test_ordinary.zdict", "atest.dict", "ztest.dict"]
for d_name in d_names: for d_name in d_names:
node.query("CREATE DICTIONARY {}(x UInt64, y UInt64) PRIMARY KEY x " \ node.query("CREATE DICTIONARY {}(x UInt64, y UInt64) PRIMARY KEY x " \
"SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'test.dict' DB 'dict_db')) " \ "SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'test_ordinary.dict' DB 'dict_db')) " \
"LAYOUT(FLAT()) LIFETIME(0)".format(d_name)) "LAYOUT(FLAT()) LIFETIME(0)".format(d_name))
def check(): def check():

View File

@ -15,7 +15,7 @@ cluster_param = pytest.mark.parametrize("cluster", [
def started_cluster(): def started_cluster():
try: try:
cluster.start() cluster.start()
node.query("create database test engine=Ordinary") node.query("create database test")
yield cluster yield cluster
finally: finally:

View File

@ -17,7 +17,7 @@ node = cluster.add_instance('node',
def start_cluster(): def start_cluster():
try: try:
cluster.start() cluster.start()
node.query('CREATE DATABASE test ENGINE=Ordinary') node.query('CREATE DATABASE test ENGINE=Ordinary') # Different paths with Atomic
yield cluster yield cluster
finally: finally:
cluster.shutdown() cluster.shutdown()

View File

@ -27,3 +27,19 @@ def test_file_path_escaping(started_cluster):
node.exec_in_container(["bash", "-c", "test -f /var/lib/clickhouse/data/test/T%2Ea_b%2Cl%2De%21/1_1_1_0/%7EId.bin"]) node.exec_in_container(["bash", "-c", "test -f /var/lib/clickhouse/data/test/T%2Ea_b%2Cl%2De%21/1_1_1_0/%7EId.bin"])
node.exec_in_container( node.exec_in_container(
["bash", "-c", "test -f /var/lib/clickhouse/shadow/1/data/test/T%2Ea_b%2Cl%2De%21/1_1_1_0/%7EId.bin"]) ["bash", "-c", "test -f /var/lib/clickhouse/shadow/1/data/test/T%2Ea_b%2Cl%2De%21/1_1_1_0/%7EId.bin"])
def test_file_path_escaping_atomic_db(started_cluster):
node.query('CREATE DATABASE IF NOT EXISTS `test 2` ENGINE = Atomic')
node.query('''
CREATE TABLE `test 2`.`T.a_b,l-e!` UUID '12345678-1000-4000-8000-000000000001' (`~Id` UInt32)
ENGINE = MergeTree() PARTITION BY `~Id` ORDER BY `~Id` SETTINGS min_bytes_for_wide_part = 0;
''')
node.query('''INSERT INTO `test 2`.`T.a_b,l-e!` VALUES (1);''')
node.query('''ALTER TABLE `test 2`.`T.a_b,l-e!` FREEZE;''')
node.exec_in_container(["bash", "-c", "test -f /var/lib/clickhouse/store/123/12345678-1000-4000-8000-000000000001/1_1_1_0/%7EId.bin"])
# Check symlink
node.exec_in_container(["bash", "-c", "test -L /var/lib/clickhouse/data/test%202/T%2Ea_b%2Cl%2De%21"])
node.exec_in_container(["bash", "-c", "test -f /var/lib/clickhouse/data/test%202/T%2Ea_b%2Cl%2De%21/1_1_1_0/%7EId.bin"])
node.exec_in_container(
["bash", "-c", "test -f /var/lib/clickhouse/shadow/2/store/123/12345678-1000-4000-8000-000000000001/1_1_1_0/%7EId.bin"])

View File

@ -13,7 +13,7 @@ path_to_data = '/var/lib/clickhouse/'
def started_cluster(): def started_cluster():
try: try:
cluster.start() cluster.start()
q('CREATE DATABASE test ENGINE = Ordinary') q('CREATE DATABASE test ENGINE = Ordinary') # Different path in shadow/ with Atomic
yield cluster yield cluster

View File

@ -515,7 +515,7 @@ 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('CREATE DATABASE test_index ENGINE=Ordinary') # Different paths with Atomic
node1.query(''' node1.query('''
CREATE TABLE test_index.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

View File

@ -21,7 +21,7 @@ node2 = cluster.add_instance('node2',
def started_cluster(): def started_cluster():
try: try:
cluster.start() cluster.start()
node1.query('CREATE DATABASE test ENGINE=Ordinary') node1.query('CREATE DATABASE test ENGINE=Ordinary') # Different paths with Atomic
node2.query('CREATE DATABASE test ENGINE=Ordinary') node2.query('CREATE DATABASE test ENGINE=Ordinary')
yield cluster yield cluster

View File

@ -1,14 +1,14 @@
=DICTIONARY in Ordinary DB =DICTIONARY in Ordinary DB
CREATE DICTIONARY ordinary_db.dict1\n(\n `key_column` UInt64 DEFAULT 0,\n `second_column` UInt8 DEFAULT 1,\n `third_column` String DEFAULT \'qqq\'\n)\nPRIMARY KEY key_column\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'database_for_dict\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT()) CREATE DICTIONARY db_01018.dict1\n(\n `key_column` UInt64 DEFAULT 0,\n `second_column` UInt8 DEFAULT 1,\n `third_column` String DEFAULT \'qqq\'\n)\nPRIMARY KEY key_column\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'database_for_dict_01018\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT())
dict1 dict1
1 1
ordinary_db dict1 db_01018 dict1
==DETACH DICTIONARY ==DETACH DICTIONARY
0 0
==ATTACH DICTIONARY ==ATTACH DICTIONARY
dict1 dict1
1 1
ordinary_db dict1 db_01018 dict1
==DROP DICTIONARY ==DROP DICTIONARY
0 0
=DICTIONARY in Memory DB =DICTIONARY in Memory DB

View File

@ -1,12 +1,12 @@
SET send_logs_level = 'fatal'; SET send_logs_level = 'fatal';
DROP DATABASE IF EXISTS database_for_dict; DROP DATABASE IF EXISTS database_for_dict_01018;
CREATE DATABASE database_for_dict Engine = Ordinary; CREATE DATABASE database_for_dict_01018;
DROP TABLE IF EXISTS database_for_dict.table_for_dict; DROP TABLE IF EXISTS database_for_dict_01018.table_for_dict;
CREATE TABLE database_for_dict.table_for_dict CREATE TABLE database_for_dict_01018.table_for_dict
( (
key_column UInt64, key_column UInt64,
second_column UInt8, second_column UInt8,
@ -15,64 +15,64 @@ CREATE TABLE database_for_dict.table_for_dict
ENGINE = MergeTree() ENGINE = MergeTree()
ORDER BY key_column; ORDER BY key_column;
INSERT INTO database_for_dict.table_for_dict VALUES (1, 100, 'Hello world'); INSERT INTO database_for_dict_01018.table_for_dict VALUES (1, 100, 'Hello world');
DROP DATABASE IF EXISTS ordinary_db; DROP DATABASE IF EXISTS db_01018;
CREATE DATABASE ordinary_db ENGINE = Ordinary; CREATE DATABASE db_01018;
SELECT '=DICTIONARY in Ordinary DB'; SELECT '=DICTIONARY in Ordinary DB';
DROP DICTIONARY IF EXISTS ordinary_db.dict1; DROP DICTIONARY IF EXISTS db_01018.dict1;
CREATE DICTIONARY ordinary_db.dict1 CREATE DICTIONARY db_01018.dict1
( (
key_column UInt64 DEFAULT 0, key_column UInt64 DEFAULT 0,
second_column UInt8 DEFAULT 1, second_column UInt8 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_dict' PASSWORD '' DB 'database_for_dict')) SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict_01018'))
LIFETIME(MIN 1 MAX 10) LIFETIME(MIN 1 MAX 10)
LAYOUT(FLAT()); LAYOUT(FLAT());
SHOW CREATE DICTIONARY ordinary_db.dict1; SHOW CREATE DICTIONARY db_01018.dict1;
SHOW DICTIONARIES FROM ordinary_db LIKE 'dict1'; SHOW DICTIONARIES FROM db_01018 LIKE 'dict1';
EXISTS DICTIONARY ordinary_db.dict1; EXISTS DICTIONARY db_01018.dict1;
SELECT database, name FROM system.dictionaries WHERE name LIKE 'dict1'; SELECT database, name FROM system.dictionaries WHERE name LIKE 'dict1';
SELECT '==DETACH DICTIONARY'; SELECT '==DETACH DICTIONARY';
DETACH DICTIONARY ordinary_db.dict1; DETACH DICTIONARY db_01018.dict1;
SHOW DICTIONARIES FROM ordinary_db LIKE 'dict1'; SHOW DICTIONARIES FROM db_01018 LIKE 'dict1';
EXISTS DICTIONARY ordinary_db.dict1; EXISTS DICTIONARY db_01018.dict1;
SELECT database, name FROM system.dictionaries WHERE name LIKE 'dict1'; SELECT database, name FROM system.dictionaries WHERE name LIKE 'dict1';
SELECT '==ATTACH DICTIONARY'; SELECT '==ATTACH DICTIONARY';
ATTACH DICTIONARY ordinary_db.dict1; ATTACH DICTIONARY db_01018.dict1;
SHOW DICTIONARIES FROM ordinary_db LIKE 'dict1'; SHOW DICTIONARIES FROM db_01018 LIKE 'dict1';
EXISTS DICTIONARY ordinary_db.dict1; EXISTS DICTIONARY db_01018.dict1;
SELECT database, name FROM system.dictionaries WHERE name LIKE 'dict1'; SELECT database, name FROM system.dictionaries WHERE name LIKE 'dict1';
SELECT '==DROP DICTIONARY'; SELECT '==DROP DICTIONARY';
DROP DICTIONARY IF EXISTS ordinary_db.dict1; DROP DICTIONARY IF EXISTS db_01018.dict1;
SHOW DICTIONARIES FROM ordinary_db LIKE 'dict1'; SHOW DICTIONARIES FROM db_01018 LIKE 'dict1';
EXISTS DICTIONARY ordinary_db.dict1; EXISTS DICTIONARY db_01018.dict1;
SELECT database, name FROM system.dictionaries WHERE name LIKE 'dict1'; SELECT database, name FROM system.dictionaries WHERE name LIKE 'dict1';
DROP DATABASE IF EXISTS ordinary_db; DROP DATABASE IF EXISTS db_01018;
DROP DATABASE IF EXISTS memory_db; DROP DATABASE IF EXISTS memory_db;
@ -87,7 +87,7 @@ CREATE DICTIONARY memory_db.dict2
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_dict' PASSWORD '' DB 'database_for_dict')) SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict_01018'))
LIFETIME(MIN 1 MAX 10) LIFETIME(MIN 1 MAX 10)
LAYOUT(FLAT()); -- {serverError 48} LAYOUT(FLAT()); -- {serverError 48}
@ -112,7 +112,7 @@ CREATE DICTIONARY lazy_db.dict3
third_column String DEFAULT 'qqq' third_column String DEFAULT 'qqq'
) )
PRIMARY KEY key_column, second_column PRIMARY KEY key_column, second_column
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict')) SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict_01018'))
LIFETIME(MIN 1 MAX 10) LIFETIME(MIN 1 MAX 10)
LAYOUT(COMPLEX_KEY_HASHED()); -- {serverError 48} LAYOUT(COMPLEX_KEY_HASHED()); -- {serverError 48}
@ -120,45 +120,45 @@ DROP DATABASE IF EXISTS lazy_db;
SELECT '=DROP DATABASE WITH DICTIONARY'; SELECT '=DROP DATABASE WITH DICTIONARY';
DROP DATABASE IF EXISTS ordinary_db; DROP DATABASE IF EXISTS db_01018;
CREATE DATABASE ordinary_db ENGINE = Ordinary; CREATE DATABASE db_01018;
CREATE DICTIONARY ordinary_db.dict4 CREATE DICTIONARY db_01018.dict4
( (
key_column UInt64 DEFAULT 0, key_column UInt64 DEFAULT 0,
second_column UInt8 DEFAULT 1, second_column UInt8 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_dict' PASSWORD '' DB 'database_for_dict')) SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict_01018'))
LIFETIME(MIN 1 MAX 10) LIFETIME(MIN 1 MAX 10)
LAYOUT(FLAT()); LAYOUT(FLAT());
SHOW DICTIONARIES FROM ordinary_db; SHOW DICTIONARIES FROM db_01018;
DROP DATABASE IF EXISTS ordinary_db; DROP DATABASE IF EXISTS db_01018;
CREATE DATABASE ordinary_db ENGINE = Ordinary; CREATE DATABASE db_01018;
SHOW DICTIONARIES FROM ordinary_db; SHOW DICTIONARIES FROM db_01018;
CREATE DICTIONARY ordinary_db.dict4 CREATE DICTIONARY db_01018.dict4
( (
key_column UInt64 DEFAULT 0, key_column UInt64 DEFAULT 0,
second_column UInt8 DEFAULT 1, second_column UInt8 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_dict' PASSWORD '' DB 'database_for_dict')) SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict_01018'))
LIFETIME(MIN 1 MAX 10) LIFETIME(MIN 1 MAX 10)
LAYOUT(FLAT()); LAYOUT(FLAT());
SHOW DICTIONARIES FROM ordinary_db; SHOW DICTIONARIES FROM db_01018;
DROP DATABASE IF EXISTS ordinary_db; DROP DATABASE IF EXISTS db_01018;
DROP TABLE IF EXISTS database_for_dict.table_for_dict; DROP TABLE IF EXISTS database_for_dict_01018.table_for_dict;
DROP DATABASE IF EXISTS database_for_dict; DROP DATABASE IF EXISTS database_for_dict_01018;
DROP DATABASE IF EXISTS memory_db; DROP DATABASE IF EXISTS memory_db;

View File

@ -2,7 +2,7 @@ SET send_logs_level = 'fatal';
DROP DATABASE IF EXISTS database_for_dict; DROP DATABASE IF EXISTS database_for_dict;
CREATE DATABASE database_for_dict Engine = Ordinary; CREATE DATABASE database_for_dict;
CREATE TABLE database_for_dict.table_for_dict CREATE TABLE database_for_dict.table_for_dict
( (

View File

@ -2,7 +2,7 @@ SET send_logs_level = 'fatal';
DROP DATABASE IF EXISTS database_for_dict; DROP DATABASE IF EXISTS database_for_dict;
CREATE DATABASE database_for_dict Engine = Ordinary; CREATE DATABASE database_for_dict;
SELECT '***date dict***'; SELECT '***date dict***';

View File

@ -2,7 +2,7 @@ SET send_logs_level = 'fatal';
DROP DATABASE IF EXISTS database_for_dict; DROP DATABASE IF EXISTS database_for_dict;
CREATE DATABASE database_for_dict Engine = Ordinary; CREATE DATABASE database_for_dict;
CREATE TABLE database_for_dict.table_for_dict CREATE TABLE database_for_dict.table_for_dict
( (

View File

@ -2,7 +2,7 @@ SET send_logs_level = 'fatal';
DROP DATABASE IF EXISTS database_for_dict; DROP DATABASE IF EXISTS database_for_dict;
CREATE DATABASE database_for_dict Engine = Ordinary; CREATE DATABASE database_for_dict;
DROP TABLE IF EXISTS database_for_dict.table_for_dict; DROP TABLE IF EXISTS database_for_dict.table_for_dict;
@ -19,7 +19,7 @@ INSERT INTO database_for_dict.table_for_dict VALUES (1, 100, 'Hello world');
DROP DATABASE IF EXISTS ordinary_db; DROP DATABASE IF EXISTS ordinary_db;
CREATE DATABASE ordinary_db ENGINE = Ordinary; CREATE DATABASE ordinary_db;
DROP DICTIONARY IF EXISTS ordinary_db.dict1; DROP DICTIONARY IF EXISTS ordinary_db.dict1;

View File

@ -11,7 +11,7 @@ tar -xf "${CURDIR}"/01037_test_data_search.tar.gz -C "${CURDIR}"
$CLICKHOUSE_CLIENT -n --query=" $CLICKHOUSE_CLIENT -n --query="
DROP DATABASE IF EXISTS test_01037; DROP DATABASE IF EXISTS test_01037;
CREATE DATABASE test_01037 Engine = Ordinary; CREATE DATABASE test_01037;
DROP TABLE IF EXISTS test_01037.points; DROP TABLE IF EXISTS test_01037.points;
CREATE TABLE test_01037.points (x Float64, y Float64) ENGINE = Memory; CREATE TABLE test_01037.points (x Float64, y Float64) ENGINE = Memory;
" "

View File

@ -11,7 +11,7 @@ tar -xf "${CURDIR}"/01037_test_data_perf.tar.gz -C "${CURDIR}"
$CLICKHOUSE_CLIENT -n --query=" $CLICKHOUSE_CLIENT -n --query="
DROP DATABASE IF EXISTS test_01037; DROP DATABASE IF EXISTS test_01037;
CREATE DATABASE test_01037 Engine = Ordinary; CREATE DATABASE test_01037;
DROP TABLE IF EXISTS test_01037.points; DROP TABLE IF EXISTS test_01037.points;
CREATE TABLE test_01037.points (x Float64, y Float64) ENGINE = Memory; CREATE TABLE test_01037.points (x Float64, y Float64) ENGINE = Memory;
" "

View File

@ -8,7 +8,7 @@ TMP_DIR="/tmp"
$CLICKHOUSE_CLIENT -n --query=" $CLICKHOUSE_CLIENT -n --query="
DROP DATABASE IF EXISTS test_01037; DROP DATABASE IF EXISTS test_01037;
CREATE DATABASE test_01037 Engine = Ordinary; CREATE DATABASE test_01037;
DROP TABLE IF EXISTS test_01037.polygons_array; DROP TABLE IF EXISTS test_01037.polygons_array;

View File

@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS dictdb" $CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS dictdb"
$CLICKHOUSE_CLIENT --query "CREATE DATABASE dictdb Engine = Ordinary" $CLICKHOUSE_CLIENT --query "CREATE DATABASE dictdb"
$CLICKHOUSE_CLIENT --query " $CLICKHOUSE_CLIENT --query "
CREATE TABLE dictdb.dict_invalidate CREATE TABLE dictdb.dict_invalidate

View File

@ -2,7 +2,7 @@ DROP TABLE IF EXISTS dictdb.table_for_dict;
DROP DICTIONARY IF EXISTS dictdb.dict_exists; DROP DICTIONARY IF EXISTS dictdb.dict_exists;
DROP DATABASE IF EXISTS dictdb; DROP DATABASE IF EXISTS dictdb;
CREATE DATABASE dictdb ENGINE = Ordinary; CREATE DATABASE dictdb;
CREATE TABLE dictdb.table_for_dict CREATE TABLE dictdb.table_for_dict
( (

View File

@ -8,7 +8,7 @@ set -e -o pipefail
# Run the client. # Run the client.
$CLICKHOUSE_CLIENT --multiquery <<'EOF' $CLICKHOUSE_CLIENT --multiquery <<'EOF'
DROP DATABASE IF EXISTS dictdb; DROP DATABASE IF EXISTS dictdb;
CREATE DATABASE dictdb Engine = Ordinary; CREATE DATABASE dictdb;
CREATE TABLE dictdb.table(x Int64, y Int64, insert_time DateTime) ENGINE = MergeTree ORDER BY tuple(); CREATE TABLE dictdb.table(x Int64, y Int64, insert_time DateTime) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO dictdb.table VALUES (12, 102, now()); INSERT INTO dictdb.table VALUES (12, 102, now());

View File

@ -1,5 +1,5 @@
DROP DATABASE IF EXISTS dictdb; DROP DATABASE IF EXISTS dictdb;
CREATE DATABASE dictdb Engine = Ordinary; CREATE DATABASE dictdb;
CREATE TABLE dictdb.dicttbl(key Int64, value_default String, value_expression String) ENGINE = MergeTree ORDER BY tuple(); CREATE TABLE dictdb.dicttbl(key Int64, value_default String, value_expression String) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO dictdb.dicttbl VALUES (12, 'hello', '55:66:77'); INSERT INTO dictdb.dicttbl VALUES (12, 'hello', '55:66:77');

View File

@ -3,7 +3,7 @@ EXISTS TABLE db_01048.t_01048;
EXISTS DICTIONARY db_01048.t_01048; EXISTS DICTIONARY db_01048.t_01048;
DROP DATABASE IF EXISTS db_01048; DROP DATABASE IF EXISTS db_01048;
CREATE DATABASE db_01048 Engine = Ordinary; CREATE DATABASE db_01048;
DROP TABLE IF EXISTS db_01048.t_01048; DROP TABLE IF EXISTS db_01048.t_01048;
EXISTS db_01048.t_01048; EXISTS db_01048.t_01048;

View File

@ -1,5 +1,5 @@
DROP DATABASE IF EXISTS some_tests; DROP DATABASE IF EXISTS some_tests;
CREATE DATABASE some_tests ENGINE=Ordinary; CREATE DATABASE some_tests ENGINE=Ordinary; -- Different inner table name with Atomic
create table some_tests.my_table ENGINE = MergeTree(day, (day), 8192) as select today() as day, 'mystring' as str; create table some_tests.my_table ENGINE = MergeTree(day, (day), 8192) as select today() as day, 'mystring' as str;
show tables from some_tests; show tables from some_tests;

View File

@ -23,6 +23,8 @@ INSERT INTO database_for_dict.table_for_dict SELECT number, 0, -1, 'c' FROM syst
DROP DICTIONARY IF EXISTS database_for_dict.ssd_dict; DROP DICTIONARY IF EXISTS database_for_dict.ssd_dict;
-- FIXME filesystem error: in create_directory: Permission denied [/var/lib/clickhouse]
-- Probably we need rewrite it to integration test
CREATE DICTIONARY database_for_dict.ssd_dict CREATE DICTIONARY database_for_dict.ssd_dict
( (
id UInt64, id UInt64,

View File

@ -1,7 +1,7 @@
SET send_logs_level = 'fatal'; SET send_logs_level = 'fatal';
DROP DATABASE IF EXISTS db_01115; DROP DATABASE IF EXISTS db_01115;
CREATE DATABASE db_01115 Engine = Ordinary; CREATE DATABASE db_01115;
USE db_01115; USE db_01115;

View File

@ -1,5 +1,5 @@
DROP DATABASE IF EXISTS test_01190; DROP DATABASE IF EXISTS test_01190;
CREATE DATABASE test_01190 ENGINE=Ordinary; CREATE DATABASE test_01190 ENGINE=Ordinary; -- Full ATTACH requires UUID with Atomic
USE test_01190; USE test_01190;
CREATE TABLE test_01190.table_for_dict (key UInt64, col UInt8) ENGINE = Memory; CREATE TABLE test_01190.table_for_dict (key UInt64, col UInt8) ENGINE = Memory;

View File

@ -1,6 +1,6 @@
DROP DATABASE IF EXISTS dict_db_01224; DROP DATABASE IF EXISTS dict_db_01224;
DROP DATABASE IF EXISTS dict_db_01224_dictionary; DROP DATABASE IF EXISTS dict_db_01224_dictionary;
CREATE DATABASE dict_db_01224 ENGINE=Ordinary; CREATE DATABASE dict_db_01224 ENGINE=Ordinary; -- Different internal dictionary name with Atomic
CREATE DATABASE dict_db_01224_dictionary Engine=Dictionary; CREATE DATABASE dict_db_01224_dictionary Engine=Dictionary;
CREATE TABLE dict_db_01224.dict_data (key UInt64, val UInt64) Engine=Memory(); CREATE TABLE dict_db_01224.dict_data (key UInt64, val UInt64) Engine=Memory();

View File

@ -1,6 +1,6 @@
DROP DATABASE IF EXISTS dict_db_01225; DROP DATABASE IF EXISTS dict_db_01225;
DROP DATABASE IF EXISTS dict_db_01225_dictionary; DROP DATABASE IF EXISTS dict_db_01225_dictionary;
CREATE DATABASE dict_db_01225 ENGINE=Ordinary; CREATE DATABASE dict_db_01225 ENGINE=Ordinary; -- Different internal dictionary name with Atomic
CREATE DATABASE dict_db_01225_dictionary Engine=Dictionary; CREATE DATABASE dict_db_01225_dictionary Engine=Dictionary;
CREATE TABLE dict_db_01225.dict_data (key UInt64, val UInt64) Engine=Memory(); CREATE TABLE dict_db_01225.dict_data (key UInt64, val UInt64) Engine=Memory();

View File

@ -1,5 +1,5 @@
DROP DATABASE IF EXISTS test_01249; DROP DATABASE IF EXISTS test_01249;
CREATE DATABASE test_01249 ENGINE=Ordinary; CREATE DATABASE test_01249 ENGINE=Ordinary; -- Full ATTACH requires UUID with Atomic
USE test_01249; USE test_01249;
CREATE TABLE bloom_filter_idx_good(`u64` UInt64, `i32` Int32, `f64` Float64, `d` Decimal(10, 2), `s` String, `e` Enum8('a' = 1, 'b' = 2, 'c' = 3), `dt` Date, INDEX bloom_filter_a i32 TYPE bloom_filter(0, 1) GRANULARITY 1) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192; -- { serverError 42 } CREATE TABLE bloom_filter_idx_good(`u64` UInt64, `i32` Int32, `f64` Float64, `d` Decimal(10, 2), `s` String, `e` Enum8('a' = 1, 'b' = 2, 'c' = 3), `dt` Date, INDEX bloom_filter_a i32 TYPE bloom_filter(0, 1) GRANULARITY 1) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192; -- { serverError 42 }

View File

@ -1,5 +1,5 @@
DROP DATABASE IF EXISTS database_for_dict; DROP DATABASE IF EXISTS database_for_dict;
CREATE DATABASE database_for_dict Engine = Ordinary; CREATE DATABASE database_for_dict;
DROP TABLE IF EXISTS database_for_dict.dict_source; DROP TABLE IF EXISTS database_for_dict.dict_source;
CREATE TABLE database_for_dict.dict_source (id UInt64, parent_id UInt64, value String) ENGINE = Memory; CREATE TABLE database_for_dict.dict_source (id UInt64, parent_id UInt64, value String) ENGINE = Memory;

View File

@ -1,6 +1,6 @@
DROP DATABASE IF EXISTS database_for_dict; DROP DATABASE IF EXISTS database_for_dict;
CREATE DATABASE database_for_dict Engine = Ordinary; CREATE DATABASE database_for_dict;
DROP TABLE IF EXISTS database_for_dict.table_for_dict; DROP TABLE IF EXISTS database_for_dict.table_for_dict;
@ -17,7 +17,7 @@ INSERT INTO database_for_dict.table_for_dict VALUES (100500, 10000000, 'Hello wo
DROP DATABASE IF EXISTS ordinary_db; DROP DATABASE IF EXISTS ordinary_db;
CREATE DATABASE ordinary_db ENGINE = Ordinary; CREATE DATABASE ordinary_db;
DROP DICTIONARY IF EXISTS ordinary_db.dict1; DROP DICTIONARY IF EXISTS ordinary_db.dict1;

View File

@ -24,6 +24,8 @@ INSERT INTO database_for_dict.table_for_dict SELECT toString(number), number + 1
DROP DICTIONARY IF EXISTS database_for_dict.ssd_dict; DROP DICTIONARY IF EXISTS database_for_dict.ssd_dict;
-- FIXME filesystem error: in create_directory: Permission denied [/var/lib/clickhouse]
-- Probably we need rewrite it to integration test
CREATE DICTIONARY database_for_dict.ssd_dict CREATE DICTIONARY database_for_dict.ssd_dict
( (
k1 String, k1 String,

View File

@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
set -e set -e
$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS test_01320" $CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS test_01320"
$CLICKHOUSE_CLIENT --query "CREATE DATABASE test_01320 ENGINE=Ordinary" $CLICKHOUSE_CLIENT --query "CREATE DATABASE test_01320 ENGINE=Ordinary" # Different bahaviour of DROP with Atomic
function thread1() function thread1()
{ {
@ -26,4 +26,4 @@ timeout 10 bash -c thread2 &
wait wait
$CLICKHOUSE_CLIENT --query "DROP DATABASE test_01320" $CLICKHOUSE_CLIENT --query "DROP DATABASE test_01320" 2>&1 | grep -v "New table appeared in database being dropped or detached. Try again." || exit 0

View File

@ -1,5 +1,5 @@
DROP DATABASE IF EXISTS db_01391; DROP DATABASE IF EXISTS db_01391;
CREATE DATABASE db_01391 Engine = Ordinary; CREATE DATABASE db_01391;
USE db_01391; USE db_01391;
DROP TABLE IF EXISTS t; DROP TABLE IF EXISTS t;