Merge branch 'master' of https://github.com/ClickHouse/ClickHouse into issue-8828

This commit is contained in:
a.palagashvili 2020-11-11 14:10:02 +03:00
commit 94790526bd
8 changed files with 200 additions and 7 deletions

View File

@ -568,6 +568,8 @@ int Server::main(const std::vector<std::string> & /*args*/)
if (config->has("zookeeper"))
global_context->reloadZooKeeperIfChanged(config);
global_context->reloadAuxiliaryZooKeepersConfigIfChanged(config);
global_context->updateStorageConfiguration(*config);
},
/* already_loaded = */ true);

View File

@ -302,9 +302,11 @@ struct ContextShared
mutable std::mutex zookeeper_mutex;
mutable zkutil::ZooKeeperPtr zookeeper; /// Client for ZooKeeper.
ConfigurationPtr zookeeper_config; /// Stores zookeeper configs
mutable std::mutex auxiliary_zookeepers_mutex;
mutable std::map<String, zkutil::ZooKeeperPtr> auxiliary_zookeepers; /// Map for auxiliary ZooKeeper clients.
ConfigurationPtr auxiliary_zookeepers_config; /// Stores auxiliary zookeepers configs
String interserver_io_host; /// The host name by which this server is available for other servers.
UInt16 interserver_io_port = 0; /// and port.
@ -364,8 +366,7 @@ struct ContextShared
/// Initialized on demand (on distributed storages initialization) since Settings should be initialized
std::unique_ptr<Clusters> clusters;
ConfigurationPtr clusters_config; /// Stores updated configs
ConfigurationPtr zookeeper_config; /// Stores zookeeper configs
mutable std::mutex clusters_mutex; /// Guards clusters and clusters_config
mutable std::mutex clusters_mutex; /// Guards clusters and clusters_config
#if USE_EMBEDDED_COMPILER
std::shared_ptr<CompiledExpressionCache> compiled_expression_cache;
@ -1498,10 +1499,16 @@ zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const
auto zookeeper = shared->auxiliary_zookeepers.find(name);
if (zookeeper == shared->auxiliary_zookeepers.end())
{
if (!getConfigRef().has("auxiliary_zookeepers." + name))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown auxiliary ZooKeeper name '{}'. If it's required it can be added to the section <auxiliary_zookeepers> in config.xml", name);
const auto & config = shared->auxiliary_zookeepers_config ? *shared->auxiliary_zookeepers_config : getConfigRef();
if (!config.has("auxiliary_zookeepers." + name))
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Unknown auxiliary ZooKeeper name '{}'. If it's required it can be added to the section <auxiliary_zookeepers> in "
"config.xml",
name);
zookeeper->second = std::make_shared<zkutil::ZooKeeper>(getConfigRef(), "auxiliary_zookeepers." + name);
zookeeper
= shared->auxiliary_zookeepers.emplace(name, std::make_shared<zkutil::ZooKeeper>(config, "auxiliary_zookeepers." + name)).first;
}
else if (zookeeper->second->expired())
zookeeper->second = zookeeper->second->startNewSession();
@ -1515,17 +1522,38 @@ void Context::resetZooKeeper() const
shared->zookeeper.reset();
}
static void reloadZooKeeperIfChangedImpl(const ConfigurationPtr & config, const std::string & config_name, zkutil::ZooKeeperPtr & zk)
{
if (!zk || zk->configChanged(*config, config_name))
zk = std::make_shared<zkutil::ZooKeeper>(*config, config_name);
}
void Context::reloadZooKeeperIfChanged(const ConfigurationPtr & config) const
{
std::lock_guard lock(shared->zookeeper_mutex);
shared->zookeeper_config = config;
reloadZooKeeperIfChangedImpl(config, "zookeeper", shared->zookeeper);
}
if (!shared->zookeeper || shared->zookeeper->configChanged(*config, "zookeeper"))
void Context::reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & config)
{
std::lock_guard lock(shared->auxiliary_zookeepers_mutex);
shared->auxiliary_zookeepers_config = config;
for (auto it = shared->auxiliary_zookeepers.begin(); it != shared->auxiliary_zookeepers.end();)
{
shared->zookeeper = std::make_shared<zkutil::ZooKeeper>(*config, "zookeeper");
if (!config->has("auxiliary_zookeepers." + it->first))
it = shared->auxiliary_zookeepers.erase(it);
else
{
reloadZooKeeperIfChangedImpl(config, "auxiliary_zookeepers." + it->first, it->second);
++it;
}
}
}
bool Context::hasZooKeeper() const
{
return getConfigRef().has("zookeeper");

View File

@ -487,6 +487,9 @@ public:
std::shared_ptr<zkutil::ZooKeeper> getZooKeeper() const;
/// Same as above but return a zookeeper connection from auxiliary_zookeepers configuration entry.
std::shared_ptr<zkutil::ZooKeeper> getAuxiliaryZooKeeper(const String & name) const;
/// Set auxiliary zookeepers configuration at server starting or configuration reloading.
void reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & config);
/// Has ready or expired ZooKeeper
bool hasZooKeeper() const;
/// Reset current zookeeper session. Do not create a new one.

View File

@ -0,0 +1,31 @@
<?xml version="1.0"?>
<yandex>
<logger>
<level>trace</level>
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
<errorlog>/var/log/clickhouse-server/clickhouse-server.err.log</errorlog>
<size>1000M</size>
<count>10</count>
</logger>
<tcp_port>9000</tcp_port>
<listen_host>127.0.0.1</listen_host>
<openSSL>
<client>
<cacheSessions>true</cacheSessions>
<verificationMode>none</verificationMode>
<invalidCertificateHandler>
<name>AcceptCertificateHandler</name>
</invalidCertificateHandler>
</client>
</openSSL>
<max_concurrent_queries>500</max_concurrent_queries>
<mark_cache_size>5368709120</mark_cache_size>
<path>./clickhouse/</path>
<users_config>users.xml</users_config>
<max_table_size_to_drop>1</max_table_size_to_drop>
<max_partition_size_to_drop>1</max_partition_size_to_drop>
</yandex>

View File

@ -0,0 +1,23 @@
<?xml version="1.0"?>
<yandex>
<profiles>
<default>
</default>
</profiles>
<users>
<default>
<password></password>
<networks incl="networks" replace="replace">
<ip>::/0</ip>
</networks>
<profile>default</profile>
<quota>default</quota>
</default>
</users>
<quotas>
<default>
</default>
</quotas>
</yandex>

View File

@ -0,0 +1,17 @@
<yandex>
<zookeeper>
<node index="1">
<host>zoo1</host>
<port>2181</port>
</node>
<node index="2">
<host>zoo2</host>
<port>2181</port>
</node>
<node index="3">
<host>zoo3</host>
<port>2181</port>
</node>
<session_timeout_ms>2000</session_timeout_ms>
</zookeeper>
</yandex>

View File

@ -0,0 +1,89 @@
import time
import pytest
import os
from helpers.cluster import ClickHouseCluster
from helpers.client import QueryRuntimeException
from helpers.test_tools import assert_eq_with_retry
cluster = ClickHouseCluster(__file__, zookeeper_config_path="configs/zookeeper.xml")
node = cluster.add_instance("node", with_zookeeper=True)
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_reload_auxiliary_zookeepers(start_cluster):
node.query(
"CREATE TABLE simple (date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', 'node') ORDER BY tuple() PARTITION BY date;"
)
node.query("INSERT INTO simple VALUES ('2020-08-27', 1)")
node.query(
"CREATE TABLE simple2 (date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/1/simple', 'node') ORDER BY tuple() PARTITION BY date;"
)
# Add an auxiliary zookeeper
new_config = """<yandex>
<zookeeper>
<node index="1">
<host>zoo1</host>
<port>2181</port>
</node>
<node index="2">
<host>zoo2</host>
<port>2181</port>
</node>
<node index="3">
<host>zoo3</host>
<port>2181</port>
</node>
<session_timeout_ms>2000</session_timeout_ms>
</zookeeper>
<auxiliary_zookeepers>
<zookeeper2>
<node index="1">
<host>zoo1</host>
<port>2181</port>
</node>
<node index="2">
<host>zoo2</host>
<port>2181</port>
</node>
</zookeeper2>
</auxiliary_zookeepers>
</yandex>"""
node.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config)
# Hopefully it has finished the configuration reload
time.sleep(2)
node.query(
"ALTER TABLE simple2 FETCH PARTITION '2020-08-27' FROM 'zookeeper2:/clickhouse/tables/0/simple';"
)
node.query("ALTER TABLE simple2 ATTACH PARTITION '2020-08-27';")
assert node.query("SELECT id FROM simple2").strip() == "1"
new_config = """<yandex>
<zookeeper>
<node index="1">
<host>zoo2</host>
<port>2181</port>
</node>
<session_timeout_ms>2000</session_timeout_ms>
</zookeeper>
</yandex>"""
node.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config)
time.sleep(2)
with pytest.raises(QueryRuntimeException):
node.query(
"ALTER TABLE simple2 FETCH PARTITION '2020-08-27' FROM 'zookeeper2:/clickhouse/tables/0/simple';"
)
assert node.query("SELECT id FROM simple2").strip() == "1"