Skip to content

Commit 670e98f

Browse files
prolleralexey-milovidov
authored andcommitted
allow several <graphite> targets (#603)
* allow several <graphite> targets * fix * fix * Adjustable parts * changelog version * fix * changelog * Style fixes * attachSystemTables * config describe * fixes * fixes
1 parent daefb87 commit 670e98f

17 files changed

+212
-96
lines changed

CHANGELOG.md

+6
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,10 @@
11

2+
## [1.1.54189](https://github.com/yandex/Clickhouse/tree/v1.1.54189-testing) (2017-03-17)
3+
[Full Changelog](https://github.com/yandex/Clickhouse/compare/v1.1.54188-stable...v1.1.54189-testing)
4+
5+
- Config: Allow define several graphite blocks, graphite.interval=60 option added. use_graphite option deleted.
6+
7+
28
## [1.1.54181](https://github.com/yandex/Clickhouse/tree/v1.1.54181-testing) (2017-03-10)
39
[Full Changelog](https://github.com/yandex/Clickhouse/compare/v1.1.54165-stable...v1.1.54181-testing)
410

Original file line numberDiff line numberDiff line change
@@ -0,0 +1,16 @@
1+
#pragma once
2+
#include <string>
3+
#include <vector>
4+
5+
namespace Poco
6+
{
7+
namespace Util
8+
{
9+
class AbstractConfiguration;
10+
}
11+
}
12+
namespace DB
13+
{
14+
/// get all internal key names for given key
15+
std::vector<std::string> getMultipleKeysFromConfig(Poco::Util::AbstractConfiguration & config, const std::string & root, const std::string & name);
16+
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,13 @@
1+
#pragma once
2+
3+
#include <DB/Databases/IDatabase.h>
4+
5+
namespace DB
6+
{
7+
class Context;
8+
class AsynchronousMetrics;
9+
10+
void attachSystemTablesServer(DatabasePtr system_database, Context * global_context, bool has_zookeeper);
11+
void attachSystemTablesLocal(DatabasePtr system_database);
12+
void attachSystemTablesAsync(DatabasePtr system_database, AsynchronousMetrics & async_metrics);
13+
}

dbms/include/DB/Storages/System/attach_system_tables.h

-13
This file was deleted.
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,21 @@
1+
#include <DB/Common/getMultipleKeysFromConfig.h>
2+
3+
#include <Poco/Util/AbstractConfiguration.h>
4+
#include <DB/Common/StringUtils.h>
5+
6+
namespace DB
7+
{
8+
std::vector<std::string> getMultipleKeysFromConfig(Poco::Util::AbstractConfiguration & config, const std::string & root, const std::string & name)
9+
{
10+
std::vector<std::string> values;
11+
Poco::Util::AbstractConfiguration::Keys config_keys;
12+
config.keys(root, config_keys);
13+
for (const auto & key : config_keys)
14+
{
15+
if (key != name && !(startsWith(key.data(), name + "[") && endsWith(key.data(), "]")))
16+
continue;
17+
values.emplace_back(key);
18+
}
19+
return values;
20+
}
21+
}

dbms/src/Server/LocalServer.cpp

+2-2
Original file line numberDiff line numberDiff line change
@@ -5,7 +5,7 @@
55
#include <Poco/Util/OptionCallback.h>
66
#include <Poco/String.h>
77
#include <DB/Databases/DatabaseOrdinary.h>
8-
#include <DB/Storages/System/attach_system_tables.h>
8+
#include <DB/Storages/System/attachSystemTables.h>
99
#include <DB/Interpreters/Context.h>
1010
#include <DB/Interpreters/ProcessList.h>
1111
#include <DB/Interpreters/executeQuery.h>
@@ -379,7 +379,7 @@ void LocalServer::attachSystemTables()
379379
context->addDatabase("system", system_database);
380380
}
381381

382-
attach_system_tables_local(system_database);
382+
attachSystemTablesLocal(system_database);
383383
}
384384

385385

+42-25
Original file line numberDiff line numberDiff line change
@@ -1,15 +1,15 @@
11
#include "MetricsTransmitter.h"
22

3+
#include <Poco/Util/Application.h>
4+
#include <Poco/Util/LayeredConfiguration.h>
35
#include <daemon/BaseDaemon.h>
4-
#include <DB/Common/setThreadName.h>
56
#include <DB/Common/CurrentMetrics.h>
67
#include <DB/Common/Exception.h>
8+
#include <DB/Common/setThreadName.h>
79
#include <DB/Interpreters/AsynchronousMetrics.h>
810

9-
1011
namespace DB
1112
{
12-
1313
MetricsTransmitter::~MetricsTransmitter()
1414
{
1515
try
@@ -32,13 +32,19 @@ MetricsTransmitter::~MetricsTransmitter()
3232

3333
void MetricsTransmitter::run()
3434
{
35-
setThreadName("MetricsTransmit");
36-
37-
/// Next minute at 00 seconds. To avoid time drift and transmit values exactly each minute.
38-
const auto get_next_minute = []
39-
{
40-
return std::chrono::time_point_cast<std::chrono::minutes, std::chrono::system_clock>(
41-
std::chrono::system_clock::now() + std::chrono::minutes(1));
35+
auto & config = Poco::Util::Application::instance().config();
36+
auto interval = config.getInt(config_name + ".interval", 60);
37+
38+
const std::string thread_name = "MericsTrns " + std::to_string(interval) + "s";
39+
setThreadName(thread_name.c_str());
40+
41+
const auto get_next_time = [](size_t seconds) {
42+
/// To avoid time drift and transmit values exactly each interval:
43+
/// next time aligned to system seconds
44+
/// (60s -> every minute at 00 seconds, 5s -> every minute:[00, 05, 15 ... 55]s, 3600 -> every hour:00:00
45+
return std::chrono::system_clock::time_point(
46+
(std::chrono::duration_cast<std::chrono::seconds>(std::chrono::system_clock::now().time_since_epoch()) / seconds) * seconds
47+
+ std::chrono::seconds(seconds));
4248
};
4349

4450
std::vector<ProfileEvents::Count> prev_counters(ProfileEvents::end());
@@ -47,7 +53,7 @@ void MetricsTransmitter::run()
4753

4854
while (true)
4955
{
50-
if (cond.wait_until(lock, get_next_minute(), [this] { return quit; }))
56+
if (cond.wait_until(lock, get_next_time(interval), [this] { return quit; }))
5157
break;
5258

5359
transmit(prev_counters);
@@ -57,35 +63,46 @@ void MetricsTransmitter::run()
5763

5864
void MetricsTransmitter::transmit(std::vector<ProfileEvents::Count> & prev_counters)
5965
{
66+
auto & config = Poco::Util::Application::instance().config();
6067
auto async_metrics_values = async_metrics.getValues();
6168

6269
GraphiteWriter::KeyValueVector<ssize_t> key_vals{};
6370
key_vals.reserve(ProfileEvents::end() + CurrentMetrics::end() + async_metrics_values.size());
6471

65-
for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i)
72+
73+
if (config.getBool(config_name + ".events", true))
6674
{
67-
const auto counter = ProfileEvents::counters[i].load(std::memory_order_relaxed);
68-
const auto counter_increment = counter - prev_counters[i];
69-
prev_counters[i] = counter;
75+
for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i)
76+
{
77+
const auto counter = ProfileEvents::counters[i].load(std::memory_order_relaxed);
78+
const auto counter_increment = counter - prev_counters[i];
79+
prev_counters[i] = counter;
7080

71-
std::string key {ProfileEvents::getDescription(static_cast<ProfileEvents::Event>(i))};
72-
key_vals.emplace_back(profile_events_path_prefix + key, counter_increment);
81+
std::string key{ProfileEvents::getDescription(static_cast<ProfileEvents::Event>(i))};
82+
key_vals.emplace_back(profile_events_path_prefix + key, counter_increment);
83+
}
7384
}
7485

75-
for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i)
86+
if (config.getBool(config_name + ".metrics", true))
7687
{
77-
const auto value = CurrentMetrics::values[i].load(std::memory_order_relaxed);
88+
for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i)
89+
{
90+
const auto value = CurrentMetrics::values[i].load(std::memory_order_relaxed);
7891

79-
std::string key {CurrentMetrics::getDescription(static_cast<CurrentMetrics::Metric>(i))};
80-
key_vals.emplace_back(current_metrics_path_prefix + key, value);
92+
std::string key{CurrentMetrics::getDescription(static_cast<CurrentMetrics::Metric>(i))};
93+
key_vals.emplace_back(current_metrics_path_prefix + key, value);
94+
}
8195
}
8296

83-
for (const auto & name_value : async_metrics_values)
97+
if (config.getBool(config_name + ".asynchronous_metrics", true))
8498
{
85-
key_vals.emplace_back(asynchronous_metrics_path_prefix + name_value.first, name_value.second);
99+
for (const auto & name_value : async_metrics_values)
100+
{
101+
key_vals.emplace_back(asynchronous_metrics_path_prefix + name_value.first, name_value.second);
102+
}
86103
}
87104

88-
BaseDaemon::instance().writeToGraphite(key_vals);
105+
if (key_vals.size())
106+
BaseDaemon::instance().writeToGraphite(key_vals, config_name);
89107
}
90-
91108
}

dbms/src/Server/MetricsTransmitter.h

+10-8
Original file line numberDiff line numberDiff line change
@@ -1,16 +1,15 @@
11
#pragma once
22

3-
#include <vector>
4-
#include <thread>
5-
#include <mutex>
63
#include <condition_variable>
7-
4+
#include <mutex>
5+
#include <string>
6+
#include <thread>
7+
#include <vector>
88
#include <DB/Common/ProfileEvents.h>
99

1010

1111
namespace DB
1212
{
13-
1413
class AsynchronousMetrics;
1514

1615
/** Automatically sends
@@ -22,23 +21,26 @@ class AsynchronousMetrics;
2221
class MetricsTransmitter
2322
{
2423
public:
25-
MetricsTransmitter(const AsynchronousMetrics & async_metrics_) : async_metrics(async_metrics_) {}
24+
MetricsTransmitter(const AsynchronousMetrics & async_metrics, const std::string & config_name)
25+
: async_metrics{async_metrics}, config_name{config_name}
26+
{
27+
}
2628
~MetricsTransmitter();
2729

2830
private:
2931
void run();
3032
void transmit(std::vector<ProfileEvents::Count> & prev_counters);
3133

3234
const AsynchronousMetrics & async_metrics;
35+
const std::string config_name;
3336

3437
bool quit = false;
3538
std::mutex mutex;
3639
std::condition_variable cond;
37-
std::thread thread {&MetricsTransmitter::run, this};
40+
std::thread thread{&MetricsTransmitter::run, this};
3841

3942
static constexpr auto profile_events_path_prefix = "ClickHouse.ProfileEvents.";
4043
static constexpr auto current_metrics_path_prefix = "ClickHouse.Metrics.";
4144
static constexpr auto asynchronous_metrics_path_prefix = "ClickHouse.AsynchronousMetrics.";
4245
};
43-
4446
}

dbms/src/Server/Server.cpp

+24-21
Original file line numberDiff line numberDiff line change
@@ -13,17 +13,19 @@
1313
#include <common/ErrorHandlers.h>
1414
#include <ext/scope_guard.hpp>
1515
#include <zkutil/ZooKeeper.h>
16+
#include <zkutil/ZooKeeperNodeCache.h>
1617
#include <DB/Common/Macros.h>
1718
#include <DB/Common/StringUtils.h>
1819
#include <DB/Common/getFQDNOrHostName.h>
20+
#include <DB/Common/getMultipleKeysFromConfig.h>
1921
#include <DB/Databases/DatabaseOrdinary.h>
2022
#include <DB/IO/HTTPCommon.h>
2123
#include <DB/Interpreters/AsynchronousMetrics.h>
2224
#include <DB/Interpreters/ProcessList.h>
2325
#include <DB/Interpreters/loadMetadata.h>
2426
#include <DB/Storages/MergeTree/ReshardingWorker.h>
2527
#include <DB/Storages/StorageReplicatedMergeTree.h>
26-
#include <DB/Storages/System/attach_system_tables.h>
28+
#include <DB/Storages/System/attachSystemTables.h>
2729
#include "ConfigReloader.h"
2830
#include "HTTPHandler.h"
2931
#include "InterserverIOHTTPHandler.h"
@@ -32,6 +34,7 @@
3234
#include "StatusFile.h"
3335
#include "TCPHandler.h"
3436

37+
3538
namespace DB
3639
{
3740
namespace ErrorCodes
@@ -221,7 +224,7 @@ int Server::main(const std::vector<std::string> & args)
221224
{
222225
auto old_configuration = loaded_config.configuration;
223226
loaded_config = ConfigProcessor().loadConfigWithZooKeeperIncludes(
224-
config_path, main_config_zk_node_cache, /* fallback_to_preprocessed = */ true);
227+
config_path, main_config_zk_node_cache, /* fallback_to_preprocessed = */ true);
225228
config().removeConfiguration(old_configuration.get());
226229
config().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false);
227230
}
@@ -321,11 +324,11 @@ int Server::main(const std::vector<std::string> & args)
321324

322325
/// Initialize main config reloader.
323326
std::string include_from_path = config().getString("include_from", "/etc/metrika.xml");
324-
auto main_config_reloader = std::make_unique<ConfigReloader>(
325-
config_path, include_from_path,
326-
std::move(main_config_zk_node_cache),
327-
[&](ConfigurationPtr config) { global_context->setClustersConfig(config); },
328-
/* already_loaded = */ true);
327+
auto main_config_reloader = std::make_unique<ConfigReloader>(config_path,
328+
include_from_path,
329+
std::move(main_config_zk_node_cache),
330+
[&](ConfigurationPtr config) { global_context->setClustersConfig(config); },
331+
/* already_loaded = */ true);
329332

330333
/// Initialize users config reloader.
331334
std::string users_config_path = config().getString("users_config", config_path);
@@ -337,11 +340,11 @@ int Server::main(const std::vector<std::string> & args)
337340
if (Poco::File(config_dir + users_config_path).exists())
338341
users_config_path = config_dir + users_config_path;
339342
}
340-
auto users_config_reloader = std::make_unique<ConfigReloader>(
341-
users_config_path, include_from_path,
342-
zkutil::ZooKeeperNodeCache([&] { return global_context->getZooKeeper(); }),
343-
[&](ConfigurationPtr config) { global_context->setUsersConfig(config); },
344-
/* already_loaded = */ false);
343+
auto users_config_reloader = std::make_unique<ConfigReloader>(users_config_path,
344+
include_from_path,
345+
zkutil::ZooKeeperNodeCache([&] { return global_context->getZooKeeper(); }),
346+
[&](ConfigurationPtr config) { global_context->setUsersConfig(config); },
347+
/* already_loaded = */ false);
345348

346349
/// Limit on total number of coucurrently executed queries.
347350
global_context->getProcessList().setMaxSize(config().getInt("max_concurrent_queries", 0));
@@ -385,7 +388,7 @@ int Server::main(const std::vector<std::string> & args)
385388

386389
DatabasePtr system_database = global_context->getDatabase("system");
387390

388-
attach_system_tables_server(system_database, global_context.get(), has_zookeeper);
391+
attachSystemTablesServer(system_database, global_context.get(), has_zookeeper);
389392

390393
bool has_resharding_worker = false;
391394
if (has_zookeeper && config().has("resharding"))
@@ -425,12 +428,8 @@ int Server::main(const std::vector<std::string> & args)
425428
std::vector<std::unique_ptr<Poco::Net::TCPServer>> servers;
426429

427430
std::vector<std::string> listen_hosts;
428-
Poco::Util::AbstractConfiguration::Keys config_keys;
429-
config().keys("", config_keys);
430-
for (const auto & key : config_keys)
431+
for (const auto & key : DB::getMultipleKeysFromConfig(config(), "", "listen_host"))
431432
{
432-
if (!startsWith(key.data(), "listen_host"))
433-
continue;
434433
listen_hosts.emplace_back(config().getString(key));
435434
}
436435

@@ -585,10 +584,14 @@ int Server::main(const std::vector<std::string> & args)
585584
/// This object will periodically calculate some metrics.
586585
AsynchronousMetrics async_metrics(*global_context);
587586

588-
attach_system_tables_async(system_database, async_metrics);
587+
attachSystemTablesAsync(system_database, async_metrics);
588+
589+
std::vector<std::unique_ptr<MetricsTransmitter>> metrics_transmitters;
590+
for (const auto & graphite_key : DB::getMultipleKeysFromConfig(config(), "", "graphite"))
591+
{
592+
metrics_transmitters.emplace_back(std::make_unique<MetricsTransmitter>(async_metrics, graphite_key));
593+
}
589594

590-
const auto metrics_transmitter
591-
= config().getBool("use_graphite", true) ? std::make_unique<MetricsTransmitter>(async_metrics) : nullptr;
592595

593596
waitForTerminationRequest();
594597
}

dbms/src/Server/config.d/no_graphite.xml

-3
This file was deleted.

0 commit comments

Comments
 (0)