Skip to content

Commit

Permalink
Support default users_config implicitly (#1188)
Browse files Browse the repository at this point in the history
- For most cases except mock test, there is no need to let user manually set user config explicitly. Use immutable default config implicitly if `users_config` does not exist in config file.

Signed-off-by: Tong Zhigao <[email protected]>
  • Loading branch information
solotzg authored Nov 2, 2020
1 parent d85d181 commit a2b9431
Show file tree
Hide file tree
Showing 9 changed files with 120 additions and 63 deletions.
2 changes: 1 addition & 1 deletion cluster_manage/flash_tools.py
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ def __init__(self, file_path):
self.service_addr = '{}:{}'.format(host, port)
self.http_addr = '{}:{}'.format(host, self.http_port)
self.tidb_status_addr = util.compute_addr_list(p['tidb_status_addr'])
flash_cluster = p['flash_cluster']
flash_cluster = p.get('flash_cluster', {})
self.cluster_master_ttl = flash_cluster.get('master_ttl', 60)
self.cluster_refresh_interval = min(
int(flash_cluster.get('refresh_interval', 20)), self.cluster_master_ttl)
Expand Down
33 changes: 18 additions & 15 deletions dbms/src/Common/Config/ConfigReloader.h
Original file line number Diff line number Diff line change
@@ -1,18 +1,23 @@
#pragma once

#include "ConfigProcessor.h"
#include <Common/ZooKeeper/Common.h>
#include <Common/ZooKeeper/ZooKeeperNodeCache.h>
#include <time.h>
#include <string>
#include <thread>
#include <mutex>

#include <condition_variable>
#include <list>
#include <mutex>
#include <set>
#include <string>
#include <thread>

#include "ConfigProcessor.h"


namespace Poco { class Logger; }
namespace Poco
{
class Logger;
}

namespace DB
{
Expand All @@ -31,24 +36,23 @@ class ConfigReloader

/** include_from_path is usually /etc/metrika.xml (i.e. value of <include_from> tag)
*/
ConfigReloader(
const std::string & path,
Updater && updater,
bool already_loaded);
ConfigReloader(const std::string & path, Updater && updater, bool already_loaded);

~ConfigReloader();
virtual ~ConfigReloader();

/// Call this method to run the backround thread.
void start();
virtual void start();

/// Reload immediately. For SYSTEM RELOAD CONFIG query.
void reload() { reloadIfNewer(/* force */ true, /* throw_on_error */ true); }

protected:
virtual void reloadIfNewer(bool force, bool throw_on_error);
Updater & getUpdater() { return updater; }

private:
void run();

void reloadIfNewer(bool force, bool throw_on_error);

struct FileWithTimestamp;

struct FilesChangesTracker
Expand All @@ -62,7 +66,6 @@ class ConfigReloader
FilesChangesTracker getNewFileList() const;

private:

static constexpr auto reload_interval = std::chrono::seconds(2);

Poco::Logger * log = &Logger::get("ConfigReloader");
Expand All @@ -79,4 +82,4 @@ class ConfigReloader
std::mutex reload_mutex;
};

}
} // namespace DB
11 changes: 9 additions & 2 deletions dbms/src/Server/ClusterManagerService.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -10,9 +10,10 @@
namespace DB
{

const std::string CLUSTER_MANAGER_PATH_KEY = "flash.flash_cluster.cluster_manager_path";
const std::string TIFLASH_PREFIX = "flash";
const std::string CLUSTER_MANAGER_PATH_KEY = TIFLASH_PREFIX + ".flash_cluster.cluster_manager_path";
const std::string BIN_NAME = "flash_cluster_manager";
const std::string TASK_INTERVAL_KEY = "flash.flash_cluster.update_rule_interval";
const std::string TASK_INTERVAL_KEY = TIFLASH_PREFIX + ".flash_cluster.update_rule_interval";

constexpr long MILLISECOND = 1000;
constexpr long INIT_DELAY = 5;
Expand Down Expand Up @@ -41,6 +42,12 @@ ClusterManagerService::ClusterManagerService(DB::Context & context_, const std::

const auto default_bin_path = conf.getString("application.dir") + "flash_cluster_manager";

if (!conf.has(TIFLASH_PREFIX))
{
LOG_WARNING(log, "TiFlash service is not specified, cluster manager can not be started");
return;
}

if (!conf.has(CLUSTER_MANAGER_PATH_KEY))
{
LOG_WARNING(log, "Binary path of cluster manager is not set, try to use default: " << default_bin_path);
Expand Down
77 changes: 77 additions & 0 deletions dbms/src/Server/ImmutableConfigReloader.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
#pragma once

#include <Common/Config/ConfigReloader.h>
#include <Common/Config/TOMLConfiguration.h>

namespace DB
{
class ImmutableConfigReloader : public ConfigReloader
{
public:
ImmutableConfigReloader(ConfigReloader::Updater && updater) : ConfigReloader("", std::move(updater), true)
{
reloadIfNewer(/* force = */ true, /* throw_on_error = */ true);
}
~ImmutableConfigReloader() {}
void start() override {}
void reloadIfNewer(bool, bool throw_on_error) override
{
auto configuration = genDefaultConfig();

std::lock_guard<std::mutex> lock(reload_mutex);

try
{
getUpdater()(configuration);
}
catch (...)
{
if (throw_on_error)
throw;
tryLogCurrentException(log, "Error updating configuration from immutable config.");
}
}

private:
ConfigurationPtr genDefaultConfig()
{
const char * DefaultConfig = "[quotas]\n"
"[quotas.default]\n"
"[quotas.default.interval]\n"
"result_rows = 0\n"
"read_rows = 0\n"
"execution_time = 0\n"
"queries = 0\n"
"errors = 0\n"
"duration = 3600\n"
"[users]\n"
"[users.readonly]\n"
"quota = \"default\"\n"
"profile = \"readonly\"\n"
"password = \"\"\n"
"[users.readonly.networks]\n"
"ip = \"::/0\"\n"
"[users.default]\n"
"quota = \"default\"\n"
"profile = \"default\"\n"
"password = \"\"\n"
"[users.default.networks]\n"
"ip = \"::/0\"\n"
"[profiles]\n"
"[profiles.readonly]\n"
"readonly = 1\n"
"[profiles.default]\n"
"load_balancing = \"random\"\n"
"use_uncompressed_cache = 0\n"
"max_memory_usage = 10000000000\n";
std::istringstream iss(DefaultConfig);
cpptoml::parser p(iss);
ConfigurationPtr configuration(new DB::TOMLConfiguration(p.parse()));
return configuration;
}

private:
std::mutex reload_mutex;
Poco::Logger * log = &Logger::get("ImmutableConfigReloader");
};
} // namespace DB
25 changes: 15 additions & 10 deletions dbms/src/Server/Server.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,6 @@

#include <AggregateFunctions/registerAggregateFunctions.h>
#include <Common/ClickHouseRevision.h>
#include <Common/Config/ConfigReloader.h>
#include <Common/CurrentMetrics.h>
#include <Common/Macros.h>
#include <Common/StringUtils/StringUtils.h>
Expand Down Expand Up @@ -54,6 +53,7 @@

#include "ClusterManagerService.h"
#include "HTTPHandlerFactory.h"
#include "ImmutableConfigReloader.h"
#include "MetricsPrometheus.h"
#include "MetricsTransmitter.h"
#include "StatusFile.h"
Expand Down Expand Up @@ -577,19 +577,24 @@ int Server::main(const std::vector<std::string> & /*args*/)
/* already_loaded = */ true);

/// Initialize users config reloader.
std::string users_config_path = config().getString("users_config", config_path);
std::string users_config_path = config().getString("users_config", String(1, '\0'));
/// If path to users' config isn't absolute, try guess its root (current) dir.
/// At first, try to find it in dir of main config, after will use current dir.
if (users_config_path.empty() || users_config_path[0] != '/')
if (users_config_path[0])
{
std::string config_dir = Poco::Path(config_path).parent().toString();
if (Poco::File(config_dir + users_config_path).exists())
users_config_path = config_dir + users_config_path;
if (users_config_path.empty() || users_config_path[0] != '/')
{
std::string config_dir = Poco::Path(config_path).parent().toString();
if (Poco::File(config_dir + users_config_path).exists())
users_config_path = config_dir + users_config_path;
}
}
auto users_config_reloader = std::make_unique<ConfigReloader>(
users_config_path,
[&](ConfigurationPtr config) { global_context->setUsersConfig(config); },
/* already_loaded = */ false);
auto users_config_reloader = users_config_path[0]
? std::make_unique<ConfigReloader>(
users_config_path,
[&](ConfigurationPtr config) { global_context->setUsersConfig(config); },
/* already_loaded = */ false)
: std::make_unique<ImmutableConfigReloader>([&](ConfigurationPtr config) { global_context->setUsersConfig(config); });

/// Reload config in SYSTEM RELOAD CONFIG query.
global_context->setConfigReloadCallback([&]() {
Expand Down
2 changes: 0 additions & 2 deletions tests/docker/config/tics_dt.toml
Original file line number Diff line number Diff line change
@@ -1,7 +1,5 @@
tmp_path = "/tmp/tiflash/data/tmp"
display_name = "TiFlash"
default_profile = "default"
users_config = "users.toml"
# specify paths used for store data, multiple path should be seperated by comma
path = "/tmp/tiflash/data/db"
capacity = "107374182400"
Expand Down
2 changes: 0 additions & 2 deletions tests/docker/config/tics_tmt.toml
Original file line number Diff line number Diff line change
@@ -1,7 +1,5 @@
tmp_path = "/tmp/tiflash/data/tmp"
display_name = "TiFlash"
default_profile = "default"
users_config = "users.toml"
# specify paths used for store data, multiple path should be seperated by comma
path = "/tmp/tiflash/data/db"
capacity = "107374182400"
Expand Down
2 changes: 0 additions & 2 deletions tests/docker/config/tiflash_dt.toml
Original file line number Diff line number Diff line change
@@ -1,7 +1,5 @@
tmp_path = "/tmp/tiflash/data/tmp"
display_name = "TiFlash"
default_profile = "default"
users_config = "users.toml"
# specify paths used for store data, multiple path should be seperated by comma
path = "/tmp/tiflash/data/db"
capacity = "10737418240"
Expand Down
29 changes: 0 additions & 29 deletions tests/docker/config/users.toml

This file was deleted.

0 comments on commit a2b9431

Please sign in to comment.