提交 15b369bd 编写于 作者: A Alexey Milovidov

More instrumentation [#METR-23272].

上级 98e36c68
......@@ -24,6 +24,12 @@
#include <ext/range.hpp>
namespace ProfileEvents
{
extern const Event RegexpCreated;
}
namespace DB
{
......@@ -445,6 +451,7 @@ namespace Regexps
if (no_capture)
flags |= OptimizedRegularExpression::RE_NO_CAPTURE;
ProfileEvents::increment(ProfileEvents::RegexpCreated);
return new Regexp{createRegexp<like>(pattern, flags)};
});
}
......
......@@ -14,82 +14,60 @@ namespace DB
*/
struct MergeTreeSettings
{
/** Merge settings. */
/** Настройки слияний. */
/** Merge settings. */
/// Determines how unbalanced merges we could do.
/// Bigger values for more unbalanced merges. It is advisable to be more than 1 / max_parts_to_merge_at_once.
/// Опеределяет, насколько разбалансированные объединения мы готовы делать.
/// Чем больше, тем более разбалансированные. Желательно, чтобы было больше, чем 1 / max_parts_to_merge_at_once.
/// Determines how unbalanced merges we could do.
/// Bigger values for more unbalanced merges. It is advisable to be more than 1 / max_parts_to_merge_at_once.
double size_ratio_coefficient_to_merge_parts = 0.25;
/// How many parts could be merges at once.
/// Labour coefficient of parts selection O(N * max_parts_to_merge_at_once).
/// Сколько за раз сливать кусков.
/// Трудоемкость выбора кусков O(N * max_parts_to_merge_at_once).
/// How many parts could be merges at once.
/// Labour coefficient of parts selection O(N * max_parts_to_merge_at_once).
size_t max_parts_to_merge_at_once = 10;
/// But while total size of parts is too small(less than this number of bytes), we could merge more parts at once.
/// This is intentionally to allow quicker merge of too small parts, which could be accumulated too quickly.
/// Но пока суммарный размер кусков слишком маленький (меньше такого количества байт), можно сливать и больше кусков за раз.
/// Это сделано, чтобы быстрее сливать очень уж маленькие куски, которых может быстро накопиться много.
/// But while total size of parts is too small(less than this number of bytes), we could merge more parts at once.
/// This is intentionally to allow quicker merge of too small parts, which could be accumulated too quickly.
size_t merge_more_parts_if_sum_bytes_is_less_than = 100 * 1024 * 1024;
size_t max_parts_to_merge_at_once_if_small = 100;
/// Parts of more than this bytes couldn't be merged at all.
/// Куски настолько большого размера объединять нельзя вообще.
/// Parts of more than this bytes couldn't be merged at all.
size_t max_bytes_to_merge_parts = 10ULL * 1024 * 1024 * 1024;
/// No more than half of threads could execute merge of parts, if at least one part more than this size in bytes.
/// Не больше половины потоков одновременно могут выполнять слияния, в которых участвует хоть один кусок хотя бы такого размера.
/// No more than half of threads could execute merge of parts, if at least one part more than this size in bytes.
size_t max_bytes_to_merge_parts_small = 250 * 1024 * 1024;
/// Parts more than this size in bytes deny to merge at all.
/// Куски настолько большого размера в сумме, объединять нельзя вообще.
/// Parts more than this size in bytes deny to merge at all.
size_t max_sum_bytes_to_merge_parts = 25ULL * 1024 * 1024 * 1024;
/// How much times we increase the coefficient at night.
/// Во столько раз ночью увеличиваем коэффициент.
/// How much times we increase the coefficient at night.
size_t merge_parts_at_night_inc = 10;
/// How many tasks of merging parts are allowed simultaneously in ReplicatedMergeTree queue.
/// Сколько заданий на слияние кусков разрешено одновременно иметь в очереди ReplicatedMergeTree.
/// How many tasks of merging parts are allowed simultaneously in ReplicatedMergeTree queue.
size_t max_replicated_merges_in_queue = 6;
/// How many seconds to keep obsolete parts.
/// Через сколько секунд удалять ненужные куски.
time_t old_parts_lifetime = 8 * 60;
/// How many seconds to keep tmp_-directories.
/// Через сколько секунд удалять tmp_-директории.
/// How many seconds to keep tmp_-directories.
time_t temporary_directories_lifetime = 86400;
/** Inserts settings. */
/** Настройки вставок. */
/** Inserts settings. */
/// If table contains at least that many active parts, artificially slow down insert into table.
/// Если в таблице хотя бы столько активных кусков, искусственно замедлять вставки в таблицу.
/// If table contains at least that many active parts, artificially slow down insert into table.
size_t parts_to_delay_insert = 150;
/// If more than this number active parts, throw 'Too much parts ...' exception
/// Если в таблице хотя бы столько активных кусков, выдавать ошибку 'Too much parts ...'
/// If more than this number active parts, throw 'Too much parts ...' exception
size_t parts_to_throw_insert = 300;
/// Max delay of inserting data into MergeTree table in seconds, if there are a lot of unmerged parts.
/// Насколько секунд можно максимально задерживать вставку в таблицу типа MergeTree, если в ней много недомердженных кусков.
/// Max delay of inserting data into MergeTree table in seconds, if there are a lot of unmerged parts.
size_t max_delay_to_insert = 200;
/** Replication settings. */
/** Настройки репликации. */
/** Replication settings. */
/// How many last blocks of hashes should be kept in ZooKeeper.
/// Для скольки последних блоков хранить хеши в ZooKeeper.
/// How many last blocks of hashes should be kept in ZooKeeper.
size_t replicated_deduplication_window = 100;
/// Keep about this number of last records in ZooKeeper log, even if they are obsolete.
/// It doesn't affect work of tables: used only to diagnose ZooKeeper log before cleaning.
/// Хранить примерно столько последних записей в логе в ZooKeeper, даже если они никому уже не нужны.
/// Не влияет на работу таблиц; используется только чтобы успеть посмотреть на лог в ZooKeeper глазами прежде, чем его очистят.
/// Keep about this number of last records in ZooKeeper log, even if they are obsolete.
/// It doesn't affect work of tables: used only to diagnose ZooKeeper log before cleaning.
size_t replicated_logs_to_keep = 100;
/// After specified amount of time passed after replication log entry creation
......@@ -99,45 +77,38 @@ struct MergeTreeSettings
time_t prefer_fetch_merged_part_time_threshold = 3600;
size_t prefer_fetch_merged_part_size_threshold = 10ULL * 1024 * 1024 * 1024;
/// Max broken parts, if more - deny automatic deletion.
/// Настройки минимального количества битых данных, при котором отказываться автоматически их удалять.
/// Max broken parts, if more - deny automatic deletion.
size_t max_suspicious_broken_parts = 10;
/// Not apply ALTER if number of files for modification(deletion, addition) more than this.
/// Не выполнять ALTER, если количество файлов для модификации (удаления, добавления) больше указанного.
/// Not apply ALTER if number of files for modification(deletion, addition) more than this.
size_t max_files_to_modify_in_alter_columns = 50;
/// Not apply ALTER, if number of files for deletion more than this.
/// Не выполнять ALTER, если количество файлов для удаления больше указанного.
size_t max_files_to_remove_in_alter_columns = 10;
/// Maximum number of errors during parts loading, while ReplicatedMergeTree still allowed to start.
/// Максимальное количество ошибок при загрузке кусков, при котором ReplicatedMergeTree соглашается запускаться.
/// Maximum number of errors during parts loading, while ReplicatedMergeTree still allowed to start.
size_t replicated_max_unexpected_parts = 3;
size_t replicated_max_unexpectedly_merged_parts = 2;
size_t replicated_max_missing_obsolete_parts = 5;
size_t replicated_max_missing_active_parts = 20;
/// If ration of wrong parts to total number of parts is less than this - allow to start anyway.
/// Если отношение количества ошибок к общему количеству кусков меньше указанного значения, то всё-равно можно запускаться.
/// If ration of wrong parts to total number of parts is less than this - allow to start anyway.
double replicated_max_ratio_of_wrong_parts = 0.05;
/// In seconds.
size_t zookeeper_session_expiration_check_period = 60;
/** Check delay of replicas settings. */
/** Настройки проверки отставания реплик. */
/// Period to check replication delay and compare with other replicas.
/// Периодичность для проверки отставания и сравнения его с другими репликами.
/// Period to check replication delay and compare with other replicas.
size_t check_delay_period = 60;
/// Minimal delay from other replicas to yield leadership. Here and further 0 means unlimited.
/// Минимальное отставание от других реплик, при котором нужно уступить лидерство. Здесь и далее, если 0 - не ограничено.
/// Minimal delay from other replicas to yield leadership. Here and further 0 means unlimited.
size_t min_relative_delay_to_yield_leadership = 120;
/// Minimal delay from other replicas to close, stop serving requests and not return Ok during status check.
/// Минимальное отставание от других реплик, при котором нужно закрыться от запросов и не выдавать Ok для проверки статуса.
/// Minimal delay from other replicas to close, stop serving requests and not return Ok during status check.
size_t min_relative_delay_to_close = 300;
/// Minimal absolute delay to close, stop serving requests and not return Ok during status check.
/// Минимальное абсолютное отставание, при котором нужно закрыться от запросов и не выдавать Ok для проверки статуса.
/// Minimal absolute delay to close, stop serving requests and not return Ok during status check.
size_t min_absolute_delay_to_close = 0;
......@@ -175,6 +146,7 @@ struct MergeTreeSettings
SET_SIZE_T(replicated_max_missing_obsolete_parts);
SET_SIZE_T(replicated_max_missing_active_parts);
SET_DOUBLE(replicated_max_ratio_of_wrong_parts);
SET_SIZE_T(zookeeper_session_expiration_check_period);
SET_SIZE_T(check_delay_period);
SET_SIZE_T(min_relative_delay_to_yield_leadership);
SET_SIZE_T(min_relative_delay_to_close);
......
......@@ -24,6 +24,9 @@
M(ReadonlyReplica) \
M(LeaderReplica) \
M(MemoryTracking) \
M(LeaderElection) \
M(EphemeralNode) \
M(ZooKeeperWatch) \
namespace CurrentMetrics
......
......@@ -90,6 +90,12 @@
M(MergeTreeDataWriterCompressedBytes) \
M(MergeTreeDataWriterBlocks) \
M(MergeTreeDataWriterBlocksAlreadySorted) \
\
M(ObsoleteEphemeralNode) \
M(CannotRemoveEphemeralNode) \
M(LeaderElectionAcquiredLeadership) \
\
M(RegexpCreated) \
namespace ProfileEvents
{
......
......@@ -50,10 +50,10 @@ void ReplicatedMergeTreeRestartingThread::run()
constexpr auto retry_period_ms = 10 * 1000;
/// Периодичность проверки истечения сессии в ZK.
time_t check_period_ms = 60 * 1000;
Int64 check_period_ms = storage.data.settings.zookeeper_session_expiration_check_period * 1000;
/// Периодичность проверки величины отставания реплики.
if (check_period_ms > static_cast<time_t>(storage.data.settings.check_delay_period) * 1000)
if (check_period_ms > static_cast<Int64>(storage.data.settings.check_delay_period) * 1000)
check_period_ms = storage.data.settings.check_delay_period * 1000;
setThreadName("ReplMTRestart");
......
......@@ -4,6 +4,19 @@
#include <functional>
#include <memory>
#include <common/logger_useful.h>
#include <DB/Common/CurrentMetrics.h>
namespace ProfileEvents
{
extern const Event ObsoleteEphemeralNode;
extern const Event LeaderElectionAcquiredLeadership;
}
namespace CurrentMetrics
{
extern const Metric LeaderElection;
}
namespace zkutil
......@@ -53,6 +66,8 @@ private:
std::atomic<bool> shutdown {false};
zkutil::EventPtr event = std::make_shared<Poco::Event>();
CurrentMetrics::Increment metric_increment{CurrentMetrics::LeaderElection};
void createNode()
{
shutdown = false;
......@@ -88,6 +103,7 @@ private:
if (brother_identifier == identifier)
{
ProfileEvents::increment(ProfileEvents::ObsoleteEphemeralNode);
LOG_WARNING(&Logger::get("LeaderElection"), "Found obsolete ephemeral node for identifier "
+ identifier + ", removing: " + brother_path);
zookeeper.tryRemoveWithRetries(brother_path);
......@@ -120,6 +136,7 @@ private:
if (it == children.begin())
{
ProfileEvents::increment(ProfileEvents::LeaderElectionAcquiredLeadership);
handler();
return;
}
......
......@@ -9,6 +9,19 @@
#include <mutex>
#include <string>
#include <common/logger_useful.h>
#include <DB/Common/ProfileEvents.h>
#include <DB/Common/CurrentMetrics.h>
namespace ProfileEvents
{
extern const Event CannotRemoveEphemeralNode;
}
namespace CurrentMetrics
{
extern const Metric EphemeralNode;
}
namespace zkutil
......@@ -432,6 +445,7 @@ public:
}
catch (const KeeperException & e)
{
ProfileEvents::increment(ProfileEvents::CannotRemoveEphemeralNode);
DB::tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
......@@ -439,6 +453,7 @@ public:
private:
std::string path;
ZooKeeper & zookeeper;
CurrentMetrics::Increment metric_increment{CurrentMetrics::EphemeralNode};
};
using EphemeralNodeHolderPtr = EphemeralNodeHolder::Ptr;
......
......@@ -18,6 +18,11 @@ namespace ProfileEvents
extern const Event ZooKeeperGetChildren;
}
namespace CurrentMetrics
{
extern const Metric ZooKeeperWatch;
}
namespace zkutil
{
......@@ -43,6 +48,7 @@ struct WatchWithEvent
/// существует все время существования WatchWithEvent
ZooKeeper & zk;
EventPtr event;
CurrentMetrics::Increment metric_increment{CurrentMetrics::ZooKeeperWatch};
WatchWithEvent(ZooKeeper & zk_, EventPtr event_) : zk(zk_), event(event_) {}
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册