mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
enable more tests
This commit is contained in:
parent
dc1cc11764
commit
7fcf20e48c
@ -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;
|
||||||
|
@ -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));
|
||||||
|
@ -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:
|
||||||
|
@ -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)
|
||||||
|
@ -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
|
||||||
|
|
||||||
|
@ -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()")
|
||||||
|
|
||||||
|
@ -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>
|
@ -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>
|
@ -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>
|
@ -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):
|
||||||
|
@ -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') "
|
||||||
|
@ -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():
|
||||||
|
@ -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:
|
||||||
|
@ -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()
|
||||||
|
@ -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"])
|
||||||
|
@ -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
|
||||||
|
|
||||||
|
@ -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
|
||||||
|
@ -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
|
||||||
|
|
||||||
|
@ -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
|
||||||
|
@ -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;
|
||||||
|
@ -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
|
||||||
(
|
(
|
||||||
|
@ -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***';
|
||||||
|
|
||||||
|
@ -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
|
||||||
(
|
(
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
@ -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;
|
||||||
"
|
"
|
||||||
|
@ -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;
|
||||||
"
|
"
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
@ -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
|
||||||
|
@ -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
|
||||||
(
|
(
|
||||||
|
@ -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());
|
||||||
|
|
||||||
|
@ -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');
|
||||||
|
@ -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;
|
||||||
|
@ -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;
|
||||||
|
@ -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,
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
@ -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;
|
||||||
|
@ -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();
|
||||||
|
@ -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();
|
||||||
|
@ -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 }
|
||||||
|
@ -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;
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
@ -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,
|
||||||
|
@ -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
|
||||||
|
@ -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;
|
||||||
|
Loading…
Reference in New Issue
Block a user