Drop replicas from dirname for internal_replication=true
Under use_compact_format_in_distributed_parts_names=1 and
internal_replication=true the server encodes all replicas for the
directory name for async INSERT into Distributed, and the directory name
looks like:
shard1_replica1,shard1_replica2,shard3_replica3
This is required for creating connections (to specific replicas only),
but in case of internal_replication=true, this can be avoided, since
this path will always includes all replicas.
This patch replaces all replicas with "_all_replicas" marker.
Note, that initial problem was that this path may overflow the NAME_MAX
if you will have more then 15 replicas, and the server will fail to
create the directory.
Also note, that changed directory name should not be a problem, since:
- empty directories will be removed since #16729
- and replicas encoded in the directory name is also supported anyway.
2021-06-20 13:50:01 +00:00
|
|
|
# pylint: disable=redefined-outer-name
|
|
|
|
# pylint: disable=unused-argument
|
|
|
|
# pylint: disable=line-too-long
|
2020-01-05 19:36:14 +00:00
|
|
|
|
Drop replicas from dirname for internal_replication=true
Under use_compact_format_in_distributed_parts_names=1 and
internal_replication=true the server encodes all replicas for the
directory name for async INSERT into Distributed, and the directory name
looks like:
shard1_replica1,shard1_replica2,shard3_replica3
This is required for creating connections (to specific replicas only),
but in case of internal_replication=true, this can be avoided, since
this path will always includes all replicas.
This patch replaces all replicas with "_all_replicas" marker.
Note, that initial problem was that this path may overflow the NAME_MAX
if you will have more then 15 replicas, and the server will fail to
create the directory.
Also note, that changed directory name should not be a problem, since:
- empty directories will be removed since #16729
- and replicas encoded in the directory name is also supported anyway.
2021-06-20 13:50:01 +00:00
|
|
|
import pytest
|
2020-01-05 19:36:14 +00:00
|
|
|
from helpers.cluster import ClickHouseCluster
|
|
|
|
|
|
|
|
cluster = ClickHouseCluster(__file__)
|
2022-03-22 16:39:58 +00:00
|
|
|
node = cluster.add_instance("node", main_configs=["configs/remote_servers.xml"])
|
2020-01-05 19:36:14 +00:00
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
cluster_param = pytest.mark.parametrize(
|
|
|
|
"cluster",
|
|
|
|
[
|
|
|
|
("test_cluster_internal_replication"),
|
|
|
|
("test_cluster_no_internal_replication"),
|
|
|
|
],
|
|
|
|
)
|
2020-06-08 19:15:30 +00:00
|
|
|
|
2020-01-05 19:36:14 +00:00
|
|
|
|
Drop replicas from dirname for internal_replication=true
Under use_compact_format_in_distributed_parts_names=1 and
internal_replication=true the server encodes all replicas for the
directory name for async INSERT into Distributed, and the directory name
looks like:
shard1_replica1,shard1_replica2,shard3_replica3
This is required for creating connections (to specific replicas only),
but in case of internal_replication=true, this can be avoided, since
this path will always includes all replicas.
This patch replaces all replicas with "_all_replicas" marker.
Note, that initial problem was that this path may overflow the NAME_MAX
if you will have more then 15 replicas, and the server will fail to
create the directory.
Also note, that changed directory name should not be a problem, since:
- empty directories will be removed since #16729
- and replicas encoded in the directory name is also supported anyway.
2021-06-20 13:50:01 +00:00
|
|
|
def get_dist_path(cluster, table, dist_format):
|
|
|
|
if dist_format == 0:
|
2022-03-22 16:39:58 +00:00
|
|
|
return f"/var/lib/clickhouse/data/test/{table}/default@not_existing:9000"
|
|
|
|
if cluster == "test_cluster_internal_replication":
|
|
|
|
return f"/var/lib/clickhouse/data/test/{table}/shard1_all_replicas"
|
|
|
|
return f"/var/lib/clickhouse/data/test/{table}/shard1_replica1"
|
Drop replicas from dirname for internal_replication=true
Under use_compact_format_in_distributed_parts_names=1 and
internal_replication=true the server encodes all replicas for the
directory name for async INSERT into Distributed, and the directory name
looks like:
shard1_replica1,shard1_replica2,shard3_replica3
This is required for creating connections (to specific replicas only),
but in case of internal_replication=true, this can be avoided, since
this path will always includes all replicas.
This patch replaces all replicas with "_all_replicas" marker.
Note, that initial problem was that this path may overflow the NAME_MAX
if you will have more then 15 replicas, and the server will fail to
create the directory.
Also note, that changed directory name should not be a problem, since:
- empty directories will be removed since #16729
- and replicas encoded in the directory name is also supported anyway.
2021-06-20 13:50:01 +00:00
|
|
|
|
|
|
|
|
2020-01-05 19:36:14 +00:00
|
|
|
@pytest.fixture(scope="module")
|
|
|
|
def started_cluster():
|
|
|
|
try:
|
|
|
|
cluster.start()
|
2020-09-22 11:56:40 +00:00
|
|
|
node.query("create database test")
|
2020-01-05 19:36:14 +00:00
|
|
|
yield cluster
|
|
|
|
|
|
|
|
finally:
|
|
|
|
cluster.shutdown()
|
|
|
|
|
2020-09-16 04:26:10 +00:00
|
|
|
|
2020-06-08 19:15:30 +00:00
|
|
|
@cluster_param
|
|
|
|
def test_single_file(started_cluster, cluster):
|
2020-09-16 04:26:10 +00:00
|
|
|
node.query(
|
2022-03-22 16:39:58 +00:00
|
|
|
"create table test.distr_1 (x UInt64, s String) engine = Distributed('{}', database, table)".format(
|
|
|
|
cluster
|
|
|
|
)
|
|
|
|
)
|
|
|
|
node.query(
|
|
|
|
"insert into test.distr_1 values (1, 'a'), (2, 'bb'), (3, 'ccc')",
|
|
|
|
settings={"use_compact_format_in_distributed_parts_names": "1"},
|
|
|
|
)
|
2020-01-05 19:36:14 +00:00
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
path = get_dist_path(cluster, "distr_1", 1)
|
Drop replicas from dirname for internal_replication=true
Under use_compact_format_in_distributed_parts_names=1 and
internal_replication=true the server encodes all replicas for the
directory name for async INSERT into Distributed, and the directory name
looks like:
shard1_replica1,shard1_replica2,shard3_replica3
This is required for creating connections (to specific replicas only),
but in case of internal_replication=true, this can be avoided, since
this path will always includes all replicas.
This patch replaces all replicas with "_all_replicas" marker.
Note, that initial problem was that this path may overflow the NAME_MAX
if you will have more then 15 replicas, and the server will fail to
create the directory.
Also note, that changed directory name should not be a problem, since:
- empty directories will be removed since #16729
- and replicas encoded in the directory name is also supported anyway.
2021-06-20 13:50:01 +00:00
|
|
|
query = f"select * from file('{path}/1.bin', 'Distributed')"
|
2022-03-22 16:39:58 +00:00
|
|
|
out = node.exec_in_container(
|
|
|
|
["/usr/bin/clickhouse", "local", "--stacktrace", "-q", query]
|
|
|
|
)
|
2020-01-05 19:36:14 +00:00
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
assert out == "1\ta\n2\tbb\n3\tccc\n"
|
2020-01-05 19:36:14 +00:00
|
|
|
|
Drop replicas from dirname for internal_replication=true
Under use_compact_format_in_distributed_parts_names=1 and
internal_replication=true the server encodes all replicas for the
directory name for async INSERT into Distributed, and the directory name
looks like:
shard1_replica1,shard1_replica2,shard3_replica3
This is required for creating connections (to specific replicas only),
but in case of internal_replication=true, this can be avoided, since
this path will always includes all replicas.
This patch replaces all replicas with "_all_replicas" marker.
Note, that initial problem was that this path may overflow the NAME_MAX
if you will have more then 15 replicas, and the server will fail to
create the directory.
Also note, that changed directory name should not be a problem, since:
- empty directories will be removed since #16729
- and replicas encoded in the directory name is also supported anyway.
2021-06-20 13:50:01 +00:00
|
|
|
query = f"""
|
|
|
|
create table t (x UInt64, s String) engine = File('Distributed', '{path}/1.bin');
|
|
|
|
select * from t;
|
|
|
|
"""
|
2022-03-22 16:39:58 +00:00
|
|
|
out = node.exec_in_container(
|
|
|
|
["/usr/bin/clickhouse", "local", "--stacktrace", "-q", query]
|
|
|
|
)
|
2020-01-05 19:36:14 +00:00
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
assert out == "1\ta\n2\tbb\n3\tccc\n"
|
2020-01-05 19:36:14 +00:00
|
|
|
|
2020-01-28 19:39:52 +00:00
|
|
|
node.query("drop table test.distr_1")
|
2020-01-05 19:36:14 +00:00
|
|
|
|
|
|
|
|
2020-06-08 19:15:30 +00:00
|
|
|
@cluster_param
|
|
|
|
def test_two_files(started_cluster, cluster):
|
2020-09-16 04:26:10 +00:00
|
|
|
node.query(
|
2022-03-22 16:39:58 +00:00
|
|
|
"create table test.distr_2 (x UInt64, s String) engine = Distributed('{}', database, table)".format(
|
|
|
|
cluster
|
|
|
|
)
|
|
|
|
)
|
|
|
|
node.query(
|
|
|
|
"insert into test.distr_2 values (0, '_'), (1, 'a')",
|
|
|
|
settings={
|
|
|
|
"use_compact_format_in_distributed_parts_names": "1",
|
|
|
|
},
|
|
|
|
)
|
|
|
|
node.query(
|
|
|
|
"insert into test.distr_2 values (2, 'bb'), (3, 'ccc')",
|
|
|
|
settings={
|
|
|
|
"use_compact_format_in_distributed_parts_names": "1",
|
|
|
|
},
|
|
|
|
)
|
|
|
|
|
|
|
|
path = get_dist_path(cluster, "distr_2", 1)
|
Drop replicas from dirname for internal_replication=true
Under use_compact_format_in_distributed_parts_names=1 and
internal_replication=true the server encodes all replicas for the
directory name for async INSERT into Distributed, and the directory name
looks like:
shard1_replica1,shard1_replica2,shard3_replica3
This is required for creating connections (to specific replicas only),
but in case of internal_replication=true, this can be avoided, since
this path will always includes all replicas.
This patch replaces all replicas with "_all_replicas" marker.
Note, that initial problem was that this path may overflow the NAME_MAX
if you will have more then 15 replicas, and the server will fail to
create the directory.
Also note, that changed directory name should not be a problem, since:
- empty directories will be removed since #16729
- and replicas encoded in the directory name is also supported anyway.
2021-06-20 13:50:01 +00:00
|
|
|
query = f"select * from file('{path}/{{1,2,3,4}}.bin', 'Distributed') order by x"
|
2022-03-22 16:39:58 +00:00
|
|
|
out = node.exec_in_container(
|
|
|
|
["/usr/bin/clickhouse", "local", "--stacktrace", "-q", query]
|
|
|
|
)
|
2020-01-05 19:36:14 +00:00
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
assert out == "0\t_\n1\ta\n2\tbb\n3\tccc\n"
|
2020-01-05 19:36:14 +00:00
|
|
|
|
Drop replicas from dirname for internal_replication=true
Under use_compact_format_in_distributed_parts_names=1 and
internal_replication=true the server encodes all replicas for the
directory name for async INSERT into Distributed, and the directory name
looks like:
shard1_replica1,shard1_replica2,shard3_replica3
This is required for creating connections (to specific replicas only),
but in case of internal_replication=true, this can be avoided, since
this path will always includes all replicas.
This patch replaces all replicas with "_all_replicas" marker.
Note, that initial problem was that this path may overflow the NAME_MAX
if you will have more then 15 replicas, and the server will fail to
create the directory.
Also note, that changed directory name should not be a problem, since:
- empty directories will be removed since #16729
- and replicas encoded in the directory name is also supported anyway.
2021-06-20 13:50:01 +00:00
|
|
|
query = f"""
|
|
|
|
create table t (x UInt64, s String) engine = File('Distributed', '{path}/{{1,2,3,4}}.bin');
|
|
|
|
select * from t order by x;
|
|
|
|
"""
|
2022-03-22 16:39:58 +00:00
|
|
|
out = node.exec_in_container(
|
|
|
|
["/usr/bin/clickhouse", "local", "--stacktrace", "-q", query]
|
|
|
|
)
|
2020-01-05 19:36:14 +00:00
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
assert out == "0\t_\n1\ta\n2\tbb\n3\tccc\n"
|
2020-01-05 19:36:14 +00:00
|
|
|
|
2020-01-28 19:39:52 +00:00
|
|
|
node.query("drop table test.distr_2")
|
2020-03-13 18:49:46 +00:00
|
|
|
|
|
|
|
|
2020-06-08 19:15:30 +00:00
|
|
|
@cluster_param
|
|
|
|
def test_single_file_old(started_cluster, cluster):
|
2020-09-16 04:26:10 +00:00
|
|
|
node.query(
|
2022-03-22 16:39:58 +00:00
|
|
|
"create table test.distr_3 (x UInt64, s String) engine = Distributed('{}', database, table)".format(
|
|
|
|
cluster
|
|
|
|
)
|
|
|
|
)
|
|
|
|
node.query(
|
|
|
|
"insert into test.distr_3 values (1, 'a'), (2, 'bb'), (3, 'ccc')",
|
|
|
|
settings={
|
|
|
|
"use_compact_format_in_distributed_parts_names": "0",
|
|
|
|
},
|
|
|
|
)
|
2020-03-13 18:49:46 +00:00
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
path = get_dist_path(cluster, "distr_3", 0)
|
Drop replicas from dirname for internal_replication=true
Under use_compact_format_in_distributed_parts_names=1 and
internal_replication=true the server encodes all replicas for the
directory name for async INSERT into Distributed, and the directory name
looks like:
shard1_replica1,shard1_replica2,shard3_replica3
This is required for creating connections (to specific replicas only),
but in case of internal_replication=true, this can be avoided, since
this path will always includes all replicas.
This patch replaces all replicas with "_all_replicas" marker.
Note, that initial problem was that this path may overflow the NAME_MAX
if you will have more then 15 replicas, and the server will fail to
create the directory.
Also note, that changed directory name should not be a problem, since:
- empty directories will be removed since #16729
- and replicas encoded in the directory name is also supported anyway.
2021-06-20 13:50:01 +00:00
|
|
|
query = f"select * from file('{path}/1.bin', 'Distributed')"
|
2022-03-22 16:39:58 +00:00
|
|
|
out = node.exec_in_container(
|
|
|
|
["/usr/bin/clickhouse", "local", "--stacktrace", "-q", query]
|
|
|
|
)
|
2020-03-13 18:49:46 +00:00
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
assert out == "1\ta\n2\tbb\n3\tccc\n"
|
2020-03-13 18:49:46 +00:00
|
|
|
|
Drop replicas from dirname for internal_replication=true
Under use_compact_format_in_distributed_parts_names=1 and
internal_replication=true the server encodes all replicas for the
directory name for async INSERT into Distributed, and the directory name
looks like:
shard1_replica1,shard1_replica2,shard3_replica3
This is required for creating connections (to specific replicas only),
but in case of internal_replication=true, this can be avoided, since
this path will always includes all replicas.
This patch replaces all replicas with "_all_replicas" marker.
Note, that initial problem was that this path may overflow the NAME_MAX
if you will have more then 15 replicas, and the server will fail to
create the directory.
Also note, that changed directory name should not be a problem, since:
- empty directories will be removed since #16729
- and replicas encoded in the directory name is also supported anyway.
2021-06-20 13:50:01 +00:00
|
|
|
query = f"""
|
|
|
|
create table t (x UInt64, s String) engine = File('Distributed', '{path}/1.bin');
|
|
|
|
select * from t;
|
|
|
|
"""
|
2022-03-22 16:39:58 +00:00
|
|
|
out = node.exec_in_container(
|
|
|
|
["/usr/bin/clickhouse", "local", "--stacktrace", "-q", query]
|
|
|
|
)
|
2020-03-13 18:49:46 +00:00
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
assert out == "1\ta\n2\tbb\n3\tccc\n"
|
2020-03-13 18:49:46 +00:00
|
|
|
|
2020-03-18 17:38:52 +00:00
|
|
|
node.query("drop table test.distr_3")
|