mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Merge pull request #63806 from jkartseva/add-endpoint-subpath-to-plain-rw
Add `endpoint_subpath` S3 URI setting
This commit is contained in:
commit
330af1c37a
@ -139,7 +139,11 @@ namespace
|
||||
S3::URI getS3URI(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const ContextPtr & context)
|
||||
{
|
||||
String endpoint = context->getMacros()->expand(config.getString(config_prefix + ".endpoint"));
|
||||
S3::URI uri(endpoint);
|
||||
String endpoint_subpath;
|
||||
if (config.has(config_prefix + ".endpoint_subpath"))
|
||||
endpoint_subpath = context->getMacros()->expand(config.getString(config_prefix + ".endpoint_subpath"));
|
||||
|
||||
S3::URI uri(fs::path(endpoint) / endpoint_subpath);
|
||||
|
||||
/// An empty key remains empty.
|
||||
if (!uri.key.empty() && !uri.key.ends_with('/'))
|
||||
|
@ -4,6 +4,7 @@
|
||||
<disk_s3_plain_rewritable>
|
||||
<type>s3_plain_rewritable</type>
|
||||
<endpoint>http://minio1:9001/root/data/</endpoint>
|
||||
<endpoint_subpath from_env="ENDPOINT_SUBPATH"></endpoint_subpath>
|
||||
<access_key_id>minio</access_key_id>
|
||||
<secret_access_key>minio123</secret_access_key>
|
||||
</disk_s3_plain_rewritable>
|
||||
|
@ -1,24 +1,39 @@
|
||||
import pytest
|
||||
import random
|
||||
import string
|
||||
import threading
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node = cluster.add_instance(
|
||||
"node",
|
||||
main_configs=["configs/storage_conf.xml"],
|
||||
with_minio=True,
|
||||
stay_alive=True,
|
||||
)
|
||||
|
||||
insert_values = [
|
||||
"(0,'data'),(1,'data')",
|
||||
",".join(
|
||||
NUM_WORKERS = 5
|
||||
|
||||
nodes = []
|
||||
for i in range(NUM_WORKERS):
|
||||
name = "node{}".format(i + 1)
|
||||
node = cluster.add_instance(
|
||||
name,
|
||||
main_configs=["configs/storage_conf.xml"],
|
||||
env_variables={"ENDPOINT_SUBPATH": name},
|
||||
with_minio=True,
|
||||
stay_alive=True,
|
||||
)
|
||||
nodes.append(node)
|
||||
|
||||
MAX_ROWS = 1000
|
||||
|
||||
|
||||
def gen_insert_values(size):
|
||||
return ",".join(
|
||||
f"({i},'{''.join(random.choices(string.ascii_lowercase, k=5))}')"
|
||||
for i in range(10)
|
||||
),
|
||||
]
|
||||
for i in range(size)
|
||||
)
|
||||
|
||||
|
||||
insert_values = ",".join(
|
||||
f"({i},'{''.join(random.choices(string.ascii_lowercase, k=5))}')" for i in range(10)
|
||||
)
|
||||
|
||||
|
||||
@pytest.fixture(scope="module", autouse=True)
|
||||
@ -32,47 +47,71 @@ def start_cluster():
|
||||
|
||||
@pytest.mark.order(0)
|
||||
def test_insert():
|
||||
for index, value in enumerate(insert_values):
|
||||
def create_insert(node, insert_values):
|
||||
node.query(
|
||||
"""
|
||||
CREATE TABLE test_{} (
|
||||
CREATE TABLE test (
|
||||
id Int64,
|
||||
data String
|
||||
) ENGINE=MergeTree()
|
||||
ORDER BY id
|
||||
SETTINGS storage_policy='s3_plain_rewritable'
|
||||
""".format(
|
||||
index
|
||||
)
|
||||
"""
|
||||
)
|
||||
node.query("INSERT INTO test VALUES {}".format(insert_values))
|
||||
|
||||
node.query("INSERT INTO test_{} VALUES {}".format(index, value))
|
||||
insert_values_arr = [
|
||||
gen_insert_values(random.randint(1, MAX_ROWS)) for _ in range(0, NUM_WORKERS)
|
||||
]
|
||||
threads = []
|
||||
for i in range(NUM_WORKERS):
|
||||
t = threading.Thread(
|
||||
target=create_insert, args=(nodes[i], insert_values_arr[i])
|
||||
)
|
||||
threads.append(t)
|
||||
t.start()
|
||||
|
||||
for t in threads:
|
||||
t.join()
|
||||
|
||||
for i in range(NUM_WORKERS):
|
||||
assert (
|
||||
node.query("SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index))
|
||||
== value
|
||||
nodes[i].query("SELECT * FROM test ORDER BY id FORMAT Values")
|
||||
== insert_values_arr[i]
|
||||
)
|
||||
|
||||
|
||||
@pytest.mark.order(1)
|
||||
def test_restart():
|
||||
for index, value in enumerate(insert_values):
|
||||
assert (
|
||||
node.query("SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index))
|
||||
== value
|
||||
insert_values_arr = []
|
||||
for i in range(NUM_WORKERS):
|
||||
insert_values_arr.append(
|
||||
nodes[i].query("SELECT * FROM test ORDER BY id FORMAT Values")
|
||||
)
|
||||
node.restart_clickhouse()
|
||||
|
||||
for index, value in enumerate(insert_values):
|
||||
def restart(node):
|
||||
node.restart_clickhouse()
|
||||
|
||||
threads = []
|
||||
for i in range(NUM_WORKERS):
|
||||
t = threading.Thread(target=restart, args=(nodes[i],))
|
||||
threads.append(t)
|
||||
t.start()
|
||||
|
||||
for t in threads:
|
||||
t.join()
|
||||
|
||||
for i in range(NUM_WORKERS):
|
||||
assert (
|
||||
node.query("SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index))
|
||||
== value
|
||||
nodes[i].query("SELECT * FROM test ORDER BY id FORMAT Values")
|
||||
== insert_values_arr[i]
|
||||
)
|
||||
|
||||
|
||||
@pytest.mark.order(2)
|
||||
def test_drop():
|
||||
for index, value in enumerate(insert_values):
|
||||
node.query("DROP TABLE IF EXISTS test_{} SYNC".format(index))
|
||||
for i in range(NUM_WORKERS):
|
||||
nodes[i].query("DROP TABLE IF EXISTS test SYNC")
|
||||
|
||||
it = cluster.minio_client.list_objects(
|
||||
cluster.minio_bucket, "data/", recursive=True
|
||||
|
Loading…
Reference in New Issue
Block a user