mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #22434 from Avogar/catboost-reload
Update paths to the catboost model configs in config reloading
This commit is contained in:
commit
e2edecc8e2
@ -750,6 +750,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
global_context->setClustersConfig(config);
|
||||
global_context->setMacros(std::make_unique<Macros>(*config, "macros", log));
|
||||
global_context->setExternalAuthenticatorsConfig(*config);
|
||||
global_context->setExternalModelsConfig(config);
|
||||
|
||||
/// Setup protection to avoid accidental DROP for big tables (that are greater than 50 GB by default)
|
||||
if (config->has("max_table_size_to_drop"))
|
||||
@ -1302,7 +1303,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
}
|
||||
|
||||
/// try to load dictionaries immediately, throw on error and die
|
||||
ext::scope_guard dictionaries_xmls, models_xmls;
|
||||
ext::scope_guard dictionaries_xmls;
|
||||
try
|
||||
{
|
||||
if (!config().getBool("dictionaries_lazy_load", true))
|
||||
@ -1312,8 +1313,6 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
}
|
||||
dictionaries_xmls = global_context->getExternalDictionariesLoader().addConfigRepository(
|
||||
std::make_unique<ExternalLoaderXMLConfigRepository>(config(), "dictionaries_config"));
|
||||
models_xmls = global_context->getExternalModelsLoader().addConfigRepository(
|
||||
std::make_unique<ExternalLoaderXMLConfigRepository>(config(), "models_config"));
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Common/Config/AbstractConfigurationComparison.h>
|
||||
#include <Common/getMultipleKeysFromConfig.h>
|
||||
|
||||
#include <unordered_set>
|
||||
#include <common/StringRef.h>
|
||||
@ -31,6 +32,23 @@ bool isSameConfiguration(const Poco::Util::AbstractConfiguration & left, const P
|
||||
return isSameConfiguration(left, key, right, key);
|
||||
}
|
||||
|
||||
bool isSameConfigurationWithMultipleKeys(const Poco::Util::AbstractConfiguration & left, const Poco::Util::AbstractConfiguration & right, const String & root, const String & name)
|
||||
{
|
||||
if (&left == &right)
|
||||
return true;
|
||||
|
||||
auto left_multiple_keys = getMultipleKeysFromConfig(left, root, name);
|
||||
auto right_multiple_keys = getMultipleKeysFromConfig(right, root, name);
|
||||
if (left_multiple_keys.size() != right_multiple_keys.size())
|
||||
return false;
|
||||
|
||||
for (auto & key : left_multiple_keys)
|
||||
if (!isSameConfiguration(left, right, concatKeyAndSubKey(root, key)))
|
||||
return false;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool isSameConfiguration(const Poco::Util::AbstractConfiguration & left, const String & left_key,
|
||||
const Poco::Util::AbstractConfiguration & right, const String & right_key)
|
||||
{
|
||||
|
@ -13,6 +13,17 @@ namespace DB
|
||||
bool isSameConfiguration(const Poco::Util::AbstractConfiguration & left,
|
||||
const Poco::Util::AbstractConfiguration & right);
|
||||
|
||||
/// Config may have multiple keys with one name. For example:
|
||||
/// <root>
|
||||
/// <some_key>...</some_key>
|
||||
/// <some_key>...</some_key>
|
||||
/// </root>
|
||||
/// Returns true if the specified subview of the two configurations contains
|
||||
/// the same keys and values for each key with the given name.
|
||||
bool isSameConfigurationWithMultipleKeys(const Poco::Util::AbstractConfiguration & left,
|
||||
const Poco::Util::AbstractConfiguration & right,
|
||||
const String & root, const String & name);
|
||||
|
||||
/// Returns true if the specified subview of the two configurations contains the same keys and values.
|
||||
bool isSameConfiguration(const Poco::Util::AbstractConfiguration & left,
|
||||
const Poco::Util::AbstractConfiguration & right,
|
||||
|
@ -480,20 +480,15 @@ void CatBoostLibHolder::initAPI()
|
||||
|
||||
std::shared_ptr<CatBoostLibHolder> getCatBoostWrapperHolder(const std::string & lib_path)
|
||||
{
|
||||
static std::weak_ptr<CatBoostLibHolder> ptr;
|
||||
static std::shared_ptr<CatBoostLibHolder> ptr;
|
||||
static std::mutex mutex;
|
||||
|
||||
std::lock_guard lock(mutex);
|
||||
auto result = ptr.lock();
|
||||
|
||||
if (!result || result->getCurrentPath() != lib_path)
|
||||
{
|
||||
result = std::make_shared<CatBoostLibHolder>(lib_path);
|
||||
/// This assignment is not atomic, which prevents from creating lock only inside 'if'.
|
||||
ptr = result;
|
||||
}
|
||||
if (!ptr || ptr->getCurrentPath() != lib_path)
|
||||
ptr = std::make_shared<CatBoostLibHolder>(lib_path);
|
||||
|
||||
return result;
|
||||
return ptr;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -292,7 +292,7 @@ void Clusters::updateClusters(const Poco::Util::AbstractConfiguration & new_conf
|
||||
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
/// If old congig is set, remove deleted clusters from impl, otherwise just clear it.
|
||||
/// If old config is set, remove deleted clusters from impl, otherwise just clear it.
|
||||
if (old_config)
|
||||
{
|
||||
for (const auto & key : deleted_keys)
|
||||
|
@ -29,6 +29,7 @@
|
||||
#include <Disks/DiskLocal.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Interpreters/ActionLocksManager.h>
|
||||
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Core/SettingsQuirks.h>
|
||||
#include <Access/AccessControlManager.h>
|
||||
@ -225,7 +226,6 @@ private:
|
||||
void cleanThread()
|
||||
{
|
||||
setThreadName("SessionCleaner");
|
||||
|
||||
std::unique_lock lock{mutex};
|
||||
|
||||
while (true)
|
||||
@ -338,6 +338,9 @@ struct ContextSharedPart
|
||||
mutable std::optional<EmbeddedDictionaries> embedded_dictionaries; /// Metrica's dictionaries. Have lazy initialization.
|
||||
mutable std::optional<ExternalDictionariesLoader> external_dictionaries_loader;
|
||||
mutable std::optional<ExternalModelsLoader> external_models_loader;
|
||||
ConfigurationPtr external_models_config;
|
||||
ext::scope_guard models_repository_guard;
|
||||
|
||||
String default_profile_name; /// Default profile name used for default values.
|
||||
String system_profile_name; /// Profile used by system processes
|
||||
String buffer_profile_name; /// Profile used by Buffer engine for flushing to the underlying
|
||||
@ -445,6 +448,7 @@ struct ContextSharedPart
|
||||
system_logs.reset();
|
||||
embedded_dictionaries.reset();
|
||||
external_dictionaries_loader.reset();
|
||||
models_repository_guard.reset();
|
||||
external_models_loader.reset();
|
||||
buffer_flush_schedule_pool.reset();
|
||||
schedule_pool.reset();
|
||||
@ -456,7 +460,6 @@ struct ContextSharedPart
|
||||
trace_collector.reset();
|
||||
/// Stop zookeeper connection
|
||||
zookeeper.reset();
|
||||
|
||||
}
|
||||
|
||||
bool hasTraceCollector() const
|
||||
@ -1353,11 +1356,29 @@ const ExternalModelsLoader & Context::getExternalModelsLoader() const
|
||||
ExternalModelsLoader & Context::getExternalModelsLoader()
|
||||
{
|
||||
std::lock_guard lock(shared->external_models_mutex);
|
||||
return getExternalModelsLoaderUnlocked();
|
||||
}
|
||||
|
||||
ExternalModelsLoader & Context::getExternalModelsLoaderUnlocked()
|
||||
{
|
||||
if (!shared->external_models_loader)
|
||||
shared->external_models_loader.emplace(getGlobalContext());
|
||||
return *shared->external_models_loader;
|
||||
}
|
||||
|
||||
void Context::setExternalModelsConfig(const ConfigurationPtr & config, const std::string & config_name)
|
||||
{
|
||||
std::lock_guard lock(shared->external_models_mutex);
|
||||
|
||||
if (shared->external_models_config && isSameConfigurationWithMultipleKeys(*config, *shared->external_models_config, "", config_name))
|
||||
return;
|
||||
|
||||
shared->external_models_config = config;
|
||||
shared->models_repository_guard .reset();
|
||||
shared->models_repository_guard = getExternalModelsLoaderUnlocked().addConfigRepository(
|
||||
std::make_unique<ExternalLoaderXMLConfigRepository>(*config, config_name));
|
||||
}
|
||||
|
||||
|
||||
EmbeddedDictionaries & Context::getEmbeddedDictionariesImpl(const bool throw_on_error) const
|
||||
{
|
||||
|
@ -499,8 +499,11 @@ public:
|
||||
EmbeddedDictionaries & getEmbeddedDictionaries();
|
||||
ExternalDictionariesLoader & getExternalDictionariesLoader();
|
||||
ExternalModelsLoader & getExternalModelsLoader();
|
||||
ExternalModelsLoader & getExternalModelsLoaderUnlocked();
|
||||
void tryCreateEmbeddedDictionaries() const;
|
||||
|
||||
void setExternalModelsConfig(const ConfigurationPtr & config, const std::string & config_name = "models_config");
|
||||
|
||||
/// I/O formats.
|
||||
BlockInputStreamPtr getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size) const;
|
||||
|
||||
|
@ -0,0 +1,3 @@
|
||||
<yandex>
|
||||
<catboost_dynamic_library_path>/etc/clickhouse-server/model/libcatboostmodel.so</catboost_dynamic_library_path>
|
||||
</yandex>
|
@ -0,0 +1,2 @@
|
||||
<yandex>
|
||||
</yandex>
|
BIN
tests/integration/test_catboost_model_config_reload/model/libcatboostmodel.so
Executable file
BIN
tests/integration/test_catboost_model_config_reload/model/libcatboostmodel.so
Executable file
Binary file not shown.
Binary file not shown.
@ -0,0 +1,8 @@
|
||||
<models>
|
||||
<model>
|
||||
<type>catboost</type>
|
||||
<name>model1</name>
|
||||
<path>/etc/clickhouse-server/model/model.bin</path>
|
||||
<lifetime>0</lifetime>
|
||||
</model>
|
||||
</models>
|
@ -0,0 +1,8 @@
|
||||
<models>
|
||||
<model>
|
||||
<type>catboost</type>
|
||||
<name>model2</name>
|
||||
<path>/etc/clickhouse-server/model/model.bin</path>
|
||||
<lifetime>0</lifetime>
|
||||
</model>
|
||||
</models>
|
58
tests/integration/test_catboost_model_config_reload/test.py
Normal file
58
tests/integration/test_catboost_model_config_reload/test.py
Normal file
@ -0,0 +1,58 @@
|
||||
import os
|
||||
import sys
|
||||
import time
|
||||
|
||||
import pytest
|
||||
|
||||
sys.path.insert(0, os.path.dirname(os.path.dirname(os.path.abspath(__file__))))
|
||||
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node = cluster.add_instance('node', stay_alive=True, main_configs=['config/models_config.xml', 'config/catboost_lib.xml'])
|
||||
|
||||
|
||||
def copy_file_to_container(local_path, dist_path, container_id):
|
||||
os.system("docker cp {local} {cont_id}:{dist}".format(local=local_path, cont_id=container_id, dist=dist_path))
|
||||
|
||||
|
||||
config = '''<yandex>
|
||||
<models_config>/etc/clickhouse-server/model/{model_config}</models_config>
|
||||
</yandex>'''
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
|
||||
copy_file_to_container(os.path.join(SCRIPT_DIR, 'model/.'), '/etc/clickhouse-server/model', node.docker_id)
|
||||
node.restart_clickhouse()
|
||||
|
||||
yield cluster
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
def change_config(model_config):
|
||||
node.replace_config("/etc/clickhouse-server/config.d/models_config.xml", config.format(model_config=model_config))
|
||||
node.query("SYSTEM RELOAD CONFIG;")
|
||||
|
||||
|
||||
def test(started_cluster):
|
||||
# Set config with the path to the first model.
|
||||
change_config("model_config.xml")
|
||||
|
||||
node.query("SELECT modelEvaluate('model1', 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11);")
|
||||
|
||||
# Change path to the second model in config.
|
||||
change_config("model_config2.xml")
|
||||
|
||||
# Check that the new model is loaded.
|
||||
node.query("SELECT modelEvaluate('model2', 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11);")
|
||||
|
||||
# Check that the old model was unloaded.
|
||||
node.query_and_get_error("SELECT modelEvaluate('model1', 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11);")
|
||||
|
Loading…
Reference in New Issue
Block a user