ClickHouse/tests/integration/test_disk_over_web_server/test.py

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

322 lines
9.5 KiB
Python
Raw Normal View History

2021-06-13 12:56:22 +00:00
import pytest
from helpers.cluster import ClickHouseCluster
2021-08-16 14:50:07 +00:00
uuids = []
2021-06-13 12:56:22 +00:00
2021-06-13 12:56:22 +00:00
@pytest.fixture(scope="module")
def cluster():
try:
cluster = ClickHouseCluster(__file__)
cluster.add_instance(
"node1",
main_configs=["configs/storage_conf.xml"],
with_nginx=True,
allow_analyzer=False,
)
cluster.add_instance(
"node2",
main_configs=["configs/storage_conf_web.xml"],
with_nginx=True,
stay_alive=True,
with_zookeeper=True,
allow_analyzer=False,
)
cluster.add_instance(
"node3",
main_configs=["configs/storage_conf_web.xml"],
with_nginx=True,
with_zookeeper=True,
allow_analyzer=False,
)
2023-02-28 11:17:43 +00:00
cluster.add_instance(
2023-02-28 11:24:39 +00:00
"node4",
main_configs=["configs/storage_conf.xml"],
with_nginx=True,
stay_alive=True,
with_installed_binary=True,
image="clickhouse/clickhouse-server",
tag="22.8.14.53",
allow_analyzer=False,
2023-02-28 11:17:43 +00:00
)
2021-06-13 12:56:22 +00:00
cluster.start()
2021-08-16 14:50:07 +00:00
2023-02-28 11:17:43 +00:00
def create_table_and_upload_data(node, i):
node.query(
2022-09-22 14:22:05 +00:00
f"CREATE TABLE data{i} (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'def', min_bytes_for_wide_part=1;"
)
2022-09-21 19:26:55 +00:00
2023-06-27 11:40:07 +00:00
node.query("SYSTEM STOP MERGES")
2022-09-21 19:26:55 +00:00
for _ in range(10):
2023-02-28 11:17:43 +00:00
node.query(
f"INSERT INTO data{i} SELECT number FROM numbers(500000 * {i+1})"
)
2023-02-28 11:17:43 +00:00
node.query(f"SELECT * FROM data{i} ORDER BY id")
2022-09-22 14:22:05 +00:00
2023-02-28 11:17:43 +00:00
metadata_path = node.query(
f"SELECT data_paths FROM system.tables WHERE name='data{i}'"
)
metadata_path = metadata_path[
metadata_path.find("/") : metadata_path.rfind("/") + 1
]
print(f"Metadata: {metadata_path}")
2023-02-28 11:17:43 +00:00
node.exec_in_container(
[
"bash",
"-c",
"/usr/bin/clickhouse static-files-disk-uploader --test-mode --url http://nginx:80/test1 --metadata-path {}".format(
metadata_path
),
],
user="root",
)
parts = metadata_path.split("/")
print(f"UUID: {parts[3]}")
2023-02-28 11:17:43 +00:00
return parts[3]
node1 = cluster.instances["node1"]
global uuids
for i in range(2):
uuid = create_table_and_upload_data(node1, i)
uuids.append(uuid)
node4 = cluster.instances["node4"]
uuid = create_table_and_upload_data(node4, 2)
uuids.append(uuid)
2021-08-16 14:50:07 +00:00
2021-06-13 12:56:22 +00:00
yield cluster
finally:
cluster.shutdown()
@pytest.mark.parametrize("node_name", ["node2"])
def test_usage(cluster, node_name):
2021-06-18 14:13:00 +00:00
node1 = cluster.instances["node1"]
2023-02-28 11:17:43 +00:00
node4 = cluster.instances["node4"]
node2 = cluster.instances[node_name]
2021-08-16 14:50:07 +00:00
global uuids
assert len(uuids) == 3
2021-06-19 11:26:48 +00:00
for i in range(3):
node2.query(
"""
CREATE TABLE test{} UUID '{}'
2021-06-19 11:26:48 +00:00
(id Int32) ENGINE = MergeTree() ORDER BY id
SETTINGS storage_policy = 'web';
""".format(
i, uuids[i], i, i
)
)
2021-06-19 11:26:48 +00:00
result = node2.query("SELECT * FROM test{} settings max_threads=20".format(i))
2021-06-19 11:26:48 +00:00
result = node2.query("SELECT count() FROM test{}".format(i))
2022-09-22 14:22:05 +00:00
assert int(result) == 5000000 * (i + 1)
result = node2.query(
"SELECT id FROM test{} WHERE id % 56 = 3 ORDER BY id".format(i)
)
2023-02-28 11:17:43 +00:00
node = node1
if i == 2:
node = node4
assert result == node.query(
"SELECT id FROM data{} WHERE id % 56 = 3 ORDER BY id".format(i)
)
result = node2.query(
"SELECT id FROM test{} WHERE id > 789999 AND id < 999999 ORDER BY id".format(
i
)
)
2023-02-28 11:17:43 +00:00
assert result == node.query(
"SELECT id FROM data{} WHERE id > 789999 AND id < 999999 ORDER BY id".format(
i
)
)
2021-06-19 16:36:39 +00:00
2022-07-14 13:40:37 +00:00
node2.query("DROP TABLE test{} SYNC".format(i))
2021-06-19 11:26:48 +00:00
print(f"Ok {i}")
2021-08-16 14:50:07 +00:00
2021-08-23 11:26:54 +00:00
def test_incorrect_usage(cluster):
node2 = cluster.instances["node3"]
2021-08-23 11:26:54 +00:00
global uuids
node2.query(
"""
CREATE TABLE test0 UUID '{}'
2021-08-23 11:26:54 +00:00
(id Int32) ENGINE = MergeTree() ORDER BY id
SETTINGS storage_policy = 'web';
""".format(
uuids[0]
)
)
2021-08-23 11:26:54 +00:00
result = node2.query("SELECT count() FROM test0")
2022-09-22 14:22:05 +00:00
assert int(result) == 5000000
2021-08-23 11:26:54 +00:00
result = node2.query_and_get_error("ALTER TABLE test0 ADD COLUMN col1 Int32 first")
assert "Table is read-only" in result
2021-08-23 11:26:54 +00:00
result = node2.query_and_get_error("TRUNCATE TABLE test0")
assert "Table is read-only" in result
2021-08-23 11:26:54 +00:00
Fix possible SIGSEGV for web disks when file does not exists It can be triggered in multiple ways, either when file does not exists and you are trying to create MergeTree table from web (that has special code for UUID handling) or by simply OPTIMIZE TABLE FINAL for MergeTree table that is located on web disk, in both cases you will get the following: <details> <summary>stacktrace</summary> 2022.10.28 14:08:40.631226 [ 6043 ] {6165bf5f-e76b-4bca-941c-7c7ff5e3b46b} <Trace> ContextAccess (default): Access granted: OPTIMIZE ON default.data_from_web 2022.10.28 14:08:40.632017 [ 6043 ] {6165bf5f-e76b-4bca-941c-7c7ff5e3b46b} <Debug> default.data_from_web (a3e65e1f-5fd4-47ed-9dbd-307f2586b52d) (MergerMutator): Selected 1 parts from all_1_1_0 to all_1_1_0 2022.10.28 14:08:40.632496 [ 6043 ] {6165bf5f-e76b-4bca-941c-7c7ff5e3b46b} <Trace> default.data_from_web (a3e65e1f-5fd4-47ed-9dbd-307f2586b52d): Trying to reserve 1.00 MiB using storage policy from min volume index 0 2022.10.28 14:08:40.632752 [ 6043 ] {6165bf5f-e76b-4bca-941c-7c7ff5e3b46b} <Trace> DiskObjectStorage(DiskWebServer): Reserved 1.00 MiB on remote disk `web_disk`, having unreserved 16.00 EiB. 2022.10.28 14:08:40.634155 [ 6043 ] {a3e65e1f-5fd4-47ed-9dbd-307f2586b52d::all_1_1_1} <Debug> MergeTask::PrepareStage: Merging 1 parts: from all_1_1_0 to all_1_1_0 into Compact 2022.10.28 14:08:40.634498 [ 6043 ] {a3e65e1f-5fd4-47ed-9dbd-307f2586b52d::all_1_1_1} <Trace> WebObjectStorage: Loading metadata for directory: http://127.0.0.1:8080/store/a3e/a3e65e1f-5fd4-47ed-9dbd-307f2586b52d/tmp_merge_all_1_1_1 2022.10.28 14:08:40.635025 [ 6043 ] {a3e65e1f-5fd4-47ed-9dbd-307f2586b52d::all_1_1_1} <Trace> DiskWeb: Adding directory: store/a3e/a3e65e1f-5fd4-47ed-9dbd-307f2586b52d/tmp_merge_all_1_1_1/ 2022.10.28 14:08:40.635355 [ 6043 ] {a3e65e1f-5fd4-47ed-9dbd-307f2586b52d::all_1_1_1} <Trace> ReadWriteBufferFromHTTP: Sending request to http://127.0.0.1:8080/store/a3e/a3e65e1f-5fd4-47ed-9dbd-307f2586b52d/tmp_merge_all_1_1_1/.index 2022.10.28 14:08:40.639618 [ 6043 ] {a3e65e1f-5fd4-47ed-9dbd-307f2586b52d::all_1_1_1} <Trace> DiskWeb: Cannot load disk metadata. Error: Code: 86. DB::Exception: Received error from remote server /store/a3e/a3e65e1f-5fd4-47ed-9dbd-307f2586b52d/tmp_merge_all_1_1_1/.index. HTTP status code: 404 Not Found, body: <!doctype html><html><head><title>404 Not Found</title><style> body { background-color: #fcfcfc; color: #333333; margin: 0; padding:0; } h1 { font-size: 1.5em; font-weight: normal; background-color: #9999cc; min-height:2em; line-height:2em; border-bottom: 1px inset black; margin: 0; } h1, p { padding-left: 10px; } code.url { background-color: #eeeeee; font-family:monospace; padding:0 2px;} </style> </head><body><h1>Not Found</h1><p>The requested resource <code class="url">/store/a3e/a3e65e1f-5fd4-47ed-9dbd-307f2586b52d/tmp_merge_all_1_1_1/.index</code> was not found on this server.</p></body></html>: while loading disk metadata. (RECEIVED_ERROR_FROM_REMOTE_IO_SERVER) (version 22.11.1.1) 2022.10.28 14:08:40.640527 [ 5488 ] {} <Trace> BaseDaemon: Received signal 11 2022.10.28 14:08:40.641529 [ 9027 ] {} <Fatal> BaseDaemon: ######################################## 2022.10.28 14:08:40.642759 [ 9027 ] {} <Fatal> BaseDaemon: (version 22.11.1.1, build id: 12145DA78CE5E9EBB10A034177FAE5967EF81A4A) (from thread 6043) (query_id: a3e65e1f-5fd4-47ed-9dbd-307f2586b52d::all_1_1_1) (query: optimize table data_from_web final) Received signal Segmentation fault (11) 2022.10.28 14:08:40.643260 [ 9027 ] {} <Fatal> BaseDaemon: Address: NULL pointer. Access: read. Unknown si_code. 2022.10.28 14:08:40.643769 [ 9027 ] {} <Fatal> BaseDaemon: Stack trace: 0x7ffff416c0f2 0x7ffff7cd1ca8 0x7ffff679ae5e 0x7fffd52e7906 0x7fffd50c65aa 0x7fffca7a0d42 0x7fffcaee79ec 0x7fffcaf242f8 0x7fffcaf242b5 0x7fffcaf2427d 0x7fffcaf24255 0x7fffcaf2421d 0x7ffff65c3686 0x7ffff65c2295 0x7fffcaeee2a9 0x7fffcaef2c43 0x7fffcaee3c0e 0x7fffcc4a7851 0x7fffcc4a768f 0x7fffcc4abb2d 0x7fffcfdce828 0x7fffd03e3eaa 0x7fffd03dfe3b 0x7fffc8ec42d4 0x7fffc8ed51d2 0x7ffff4bdd839 0x7ffff4bde0a8 0x7ffff48ab261 0x7ffff48a769a 0x7ffff48a6335 0x7ffff409f8fd 0x7ffff4121a60 2022.10.28 14:08:40.644411 [ 9027 ] {} <Fatal> BaseDaemon: 4. ? @ 0x7ffff416c0f2 in ? 2022.10.28 14:08:40.676390 [ 9027 ] {} <Fatal> BaseDaemon: 5. /src/ch/clickhouse/src/Common/StringUtils/StringUtils.cpp:9: detail::startsWith(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>> const&, char const*, unsigned long) @ 0x1ca8 in /src/ch/clickhouse/.cmake/src/Common/StringUtils/libstring_utilsd.so 2022.10.28 14:08:40.730727 [ 9027 ] {} <Fatal> BaseDaemon: 6. /src/ch/clickhouse/src/Common/StringUtils/StringUtils.h:19: startsWith(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>> const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>> const&) @ 0x59ae5e in /src/ch/clickhouse/.cmake/src/libclickhouse_common_iod.so 2022.10.28 14:08:40.923955 [ 9027 ] {} <Fatal> BaseDaemon: 7. /src/ch/clickhouse/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.cpp:58: DB::MetadataStorageFromStaticFilesWebServer::exists(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>> const&) const @ 0x6e7906 in /src/ch/clickhouse/.cmake/src/libdbmsd.so 2022.10.28 14:08:41.291996 [ 9027 ] {} <Fatal> BaseDaemon: 8. /src/ch/clickhouse/src/Disks/ObjectStorages/DiskObjectStorage.cpp:181: DB::DiskObjectStorage::exists(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>> const&) const @ 0x4c65aa in /src/ch/clickhouse/.cmake/src/libdbmsd.so 2022.10.28 14:08:41.704697 [ 9027 ] {} <Fatal> BaseDaemon: 9. /src/ch/clickhouse/src/Storages/MergeTree/DataPartStorageOnDisk.cpp:74: DB::DataPartStorageOnDisk::exists() const @ 0xda0d42 in /src/ch/clickhouse/.cmake/src/libclickhouse_storages_mergetreed.so 2022.10.28 14:08:43.032459 [ 9027 ] {} <Fatal> BaseDaemon: 10. /src/ch/clickhouse/src/Storages/MergeTree/MergeTask.cpp:147: DB::MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() @ 0x14e79ec in /src/ch/clickhouse/.cmake/src/libclickhouse_storages_mergetreed.so ... Segmentation fault (core dumped) </details> Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-10-28 13:10:53 +00:00
result = node2.query_and_get_error("OPTIMIZE TABLE test0 FINAL")
assert "Only read-only operations are supported" in result
2022-07-14 13:40:37 +00:00
node2.query("DROP TABLE test0 SYNC")
@pytest.mark.parametrize("node_name", ["node2"])
def test_cache(cluster, node_name):
node1 = cluster.instances["node1"]
2023-02-28 11:17:43 +00:00
node4 = cluster.instances["node4"]
2022-07-14 13:40:37 +00:00
node2 = cluster.instances[node_name]
global uuids
assert len(uuids) == 3
for i in range(3):
node2.query(
"""
CREATE TABLE test{} UUID '{}'
2022-07-14 13:40:37 +00:00
(id Int32) ENGINE = MergeTree() ORDER BY id
SETTINGS storage_policy = 'cached_web';
""".format(
i, uuids[i]
2022-07-14 13:40:37 +00:00
)
)
result = node2.query(
"""
SYSTEM DROP FILESYSTEM CACHE;
SELECT count() FROM system.filesystem_cache;
"""
)
assert int(result) == 0
result = node2.query("SELECT * FROM test{} settings max_threads=20".format(i))
result = node2.query(
"""
SELECT count() FROM system.filesystem_cache;
"""
)
assert int(result) > 0
result = node2.query("SELECT count() FROM test{}".format(i))
2022-09-22 14:22:05 +00:00
assert int(result) == 5000000 * (i + 1)
2022-07-14 13:40:37 +00:00
result = node2.query(
"SELECT id FROM test{} WHERE id % 56 = 3 ORDER BY id".format(i)
)
2023-02-28 11:17:43 +00:00
node = node1
if i == 2:
node = node4
assert result == node.query(
2022-07-14 13:40:37 +00:00
"SELECT id FROM data{} WHERE id % 56 = 3 ORDER BY id".format(i)
)
result = node2.query(
"SELECT id FROM test{} WHERE id > 789999 AND id < 999999 ORDER BY id".format(
i
)
)
2023-02-28 11:17:43 +00:00
assert result == node.query(
2022-07-14 13:40:37 +00:00
"SELECT id FROM data{} WHERE id > 789999 AND id < 999999 ORDER BY id".format(
i
)
)
node2.query("DROP TABLE test{} SYNC".format(i))
print(f"Ok {i}")
def test_unavailable_server(cluster):
"""
Regression test for the case when clickhouse-server simply ignore when
server is unavailable on start and later will simply return 0 rows for
SELECT from table on web disk.
"""
node2 = cluster.instances["node2"]
global uuids
node2.query(
"""
CREATE TABLE test0 UUID '{}'
(id Int32) ENGINE = MergeTree() ORDER BY id
SETTINGS storage_policy = 'web';
""".format(
uuids[0]
)
)
node2.stop_clickhouse()
try:
# NOTE: you cannot use separate disk instead, since MergeTree engine will
# try to lookup parts on all disks (to look unexpected disks with parts)
# and fail because of unavailable server.
node2.exec_in_container(
[
"bash",
"-c",
"sed -i 's#http://nginx:80/test1/#http://nginx:8080/test1/#' /etc/clickhouse-server/config.d/storage_conf_web.xml",
]
)
with pytest.raises(Exception):
# HTTP retries with backup can take awhile
node2.start_clickhouse(start_wait_sec=120, retry_start=False)
assert node2.contains_in_log(
"Caught exception while loading metadata.*Connection refused"
)
assert node2.contains_in_log(
"HTTP request to \`http://nginx:8080/test1/.*\` failed at try 1/10 with bytes read: 0/unknown. Error: Connection refused."
)
finally:
node2.exec_in_container(
[
"bash",
"-c",
"sed -i 's#http://nginx:8080/test1/#http://nginx:80/test1/#' /etc/clickhouse-server/config.d/storage_conf_web.xml",
]
)
node2.start_clickhouse()
node2.query("DROP TABLE test0 SYNC")
def test_replicated_database(cluster):
node1 = cluster.instances["node3"]
node1.query(
"CREATE DATABASE rdb ENGINE=Replicated('/test/rdb', 's1', 'r1')",
settings={"allow_experimental_database_replicated": 1},
)
global uuids
node1.query(
"""
CREATE TABLE rdb.table0 UUID '{}'
(id Int32) ENGINE = MergeTree() ORDER BY id
SETTINGS storage_policy = 'web';
""".format(
uuids[0]
)
)
node2 = cluster.instances["node2"]
node2.query(
"CREATE DATABASE rdb ENGINE=Replicated('/test/rdb', 's1', 'r2')",
settings={"allow_experimental_database_replicated": 1},
)
node2.query("SYSTEM SYNC DATABASE REPLICA rdb")
assert node1.query("SELECT count() FROM rdb.table0") == "5000000\n"
assert node2.query("SELECT count() FROM rdb.table0") == "5000000\n"
node1.query("DROP DATABASE rdb SYNC")
node2.query("DROP DATABASE rdb SYNC")