Merge pull request #54090 from ClickHouse/remove-current-status

Remove CurrentStatusInfo
This commit is contained in:
Alexey Milovidov 2023-09-26 21:38:07 +03:00 committed by GitHub
commit 29d3b34dfc
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 0 additions and 271 deletions

View File

@ -760,27 +760,6 @@
</graphite>
-->
<!-- Serve endpoint for Prometheus monitoring. -->
<!--
endpoint - mertics path (relative to root, statring with "/")
port - port to setup server. If not defined or 0 than http_port used
metrics - send data from table system.metrics
events - send data from table system.events
asynchronous_metrics - send data from table system.asynchronous_metrics
status_info - send data from different component from CH, ex: Dictionaries status
-->
<!--
<prometheus>
<endpoint>/metrics</endpoint>
<port>9363</port>
<metrics>true</metrics>
<events>true</events>
<asynchronous_metrics>true</asynchronous_metrics>
<status_info>true</status_info>
</prometheus>
-->
<!-- Query log. Used only for queries with setting log_queries = 1. -->
<query_log>
<!-- What table to insert data. If table is not exist, it will be created.

View File

@ -638,7 +638,6 @@ default_session_timeout: 60
# metrics - send data from table system.metrics
# events - send data from table system.events
# asynchronous_metrics - send data from table system.asynchronous_metrics
# status_info - send data from different component from CH, ex: Dictionaries status
# prometheus:
# endpoint: /metrics
@ -647,7 +646,6 @@ default_session_timeout: 60
# metrics: true
# events: true
# asynchronous_metrics: true
# status_info: true
# Query log. Used only for queries with setting log_queries = 1.
query_log:

View File

@ -760,27 +760,6 @@
</graphite>
-->
<!-- Serve endpoint for Prometheus monitoring. -->
<!--
endpoint - mertics path (relative to root, statring with "/")
port - port to setup server. If not defined or 0 than http_port used
metrics - send data from table system.metrics
events - send data from table system.events
asynchronous_metrics - send data from table system.asynchronous_metrics
status_info - send data from different component from CH, ex: Dictionaries status
-->
<!--
<prometheus>
<endpoint>/metrics</endpoint>
<port>9363</port>
<metrics>true</metrics>
<events>true</events>
<asynchronous_metrics>true</asynchronous_metrics>
<status_info>true</status_info>
</prometheus>
-->
<!-- Query log. Used only for queries with setting log_queries = 1. -->
<query_log>
<!-- What table to insert data. If table is not exist, it will be created.

View File

@ -969,7 +969,6 @@
metrics - send data from table system.metrics
events - send data from table system.events
asynchronous_metrics - send data from table system.asynchronous_metrics
status_info - send data from different component from CH, ex: Dictionaries status
-->
<!--
<prometheus>
@ -979,7 +978,6 @@
<metrics>true</metrics>
<events>true</events>
<asynchronous_metrics>true</asynchronous_metrics>
<status_info>true</status_info>
</prometheus>
-->

View File

@ -639,7 +639,6 @@ default_session_timeout: 60
# metrics - send data from table system.metrics
# events - send data from table system.events
# asynchronous_metrics - send data from table system.asynchronous_metrics
# status_info - send data from different component from CH, ex: Dictionaries status
# prometheus:
# endpoint: /metrics
@ -648,7 +647,6 @@ default_session_timeout: 60
# metrics: true
# events: true
# asynchronous_metrics: true
# status_info: true
# Query log. Used only for queries with setting log_queries = 1.
query_log:

View File

@ -1,57 +0,0 @@
#include <Common/StatusInfo.h>
#include <Common/ExternalLoaderStatus.h>
/// Available status. Add something here as you wish.
#define APPLY_FOR_STATUS(M) \
M(DictionaryStatus, "Dictionary Status.", DB::getStatusEnumAllPossibleValues()) \
namespace CurrentStatusInfo
{
#define M(NAME, DOCUMENTATION, ENUM) extern const Status NAME = Status(__COUNTER__);
APPLY_FOR_STATUS(M)
#undef M
constexpr Status END = Status(__COUNTER__);
std::mutex locks[END] {};
std::unordered_map<String, Int8> values[END] {};
const char * getName(Status event)
{
static const char * strings[] =
{
#define M(NAME, DOCUMENTATION, ENUM) #NAME,
APPLY_FOR_STATUS(M)
#undef M
};
return strings[event];
}
const char * getDocumentation(Status event)
{
static const char * strings[] =
{
#define M(NAME, DOCUMENTATION, ENUM) #DOCUMENTATION,
APPLY_FOR_STATUS(M)
#undef M
};
return strings[event];
}
const std::vector<std::pair<String, Int8>> & getAllPossibleValues(Status event)
{
static const std::vector<std::pair<String, Int8>> enum_values [] =
{
#define M(NAME, DOCUMENTATION, ENUM) ENUM,
APPLY_FOR_STATUS(M)
#undef M
};
return enum_values[event];
}
Status end() { return END; }
}
#undef APPLY_FOR_STATUS

View File

@ -1,39 +0,0 @@
#pragma once
#include <stddef.h>
#include <cstdint>
#include <utility>
#include <atomic>
#include <vector>
#include <base/types.h>
#include <base/strong_typedef.h>
#include <mutex>
#include <unordered_map>
namespace CurrentStatusInfo
{
using Status = StrongTypedef<size_t, struct StatusTag>;
using Key = std::string;
const char * getName(Status event);
const char * getDocumentation(Status event);
const std::vector<std::pair<String, Int8>> & getAllPossibleValues(Status event);
extern std::unordered_map<String, Int8> values[];
extern std::mutex locks[];
Status end();
inline void set(Status status, Key key, Int8 value)
{
std::lock_guard lock(locks[status]);
values[status][key] = value;
}
inline void unset(Status status, Key key)
{
std::lock_guard lock(locks[status]);
values[status].erase(key);
}
}

View File

@ -1,7 +1,6 @@
#include "ExternalLoader.h"
#include <mutex>
#include <pcg_random.hpp>
#include <Common/MemoryTrackerBlockerInThread.h>
#include <Common/Config/AbstractConfigurationComparison.h>
#include <Common/Exception.h>
@ -9,7 +8,6 @@
#include <Common/ThreadPool.h>
#include <Common/randomSeed.h>
#include <Common/setThreadName.h>
#include <Common/StatusInfo.h>
#include <Common/scope_guard_safe.h>
#include <Common/logger_useful.h>
#include <base/chrono_io.h>
@ -18,12 +16,6 @@
#include <unordered_set>
namespace CurrentStatusInfo
{
extern const Status DictionaryStatus;
}
namespace DB
{
namespace ErrorCodes
@ -1145,7 +1137,6 @@ private:
if (info && (info->loading_id == loading_id))
{
info->loading_id = info->state_id;
CurrentStatusInfo::set(CurrentStatusInfo::DictionaryStatus, name, static_cast<Int8>(info->status()));
}
min_id_to_finish_loading_dependencies.erase(std::this_thread::get_id());
@ -1307,7 +1298,6 @@ scope_guard ExternalLoader::addConfigRepository(std::unique_ptr<IExternalLoaderC
return [this, ptr, name]()
{
config_files_reader->removeConfigRepository(ptr);
CurrentStatusInfo::unset(CurrentStatusInfo::DictionaryStatus, name);
reloadConfig(name);
};
}

View File

@ -1,7 +1,6 @@
#include "PrometheusMetricsWriter.h"
#include <IO/WriteHelpers.h>
#include <Common/StatusInfo.h>
#include <regex> /// TODO: this library is harmful.
#include <algorithm>
@ -51,7 +50,6 @@ PrometheusMetricsWriter::PrometheusMetricsWriter(
, send_events(config.getBool(config_name + ".events", true))
, send_metrics(config.getBool(config_name + ".metrics", true))
, send_asynchronous_metrics(config.getBool(config_name + ".asynchronous_metrics", true))
, send_status_info(config.getBool(config_name + ".status_info", true))
{
}
@ -120,42 +118,6 @@ void PrometheusMetricsWriter::write(WriteBuffer & wb) const
writeOutLine(wb, key, value.value);
}
}
if (send_status_info)
{
for (size_t i = 0, end = CurrentStatusInfo::end(); i < end; ++i)
{
std::lock_guard lock(CurrentStatusInfo::locks[static_cast<CurrentStatusInfo::Status>(i)]);
std::string metric_name{CurrentStatusInfo::getName(static_cast<CurrentStatusInfo::Status>(i))};
std::string metric_doc{CurrentStatusInfo::getDocumentation(static_cast<CurrentStatusInfo::Status>(i))};
convertHelpToSingleLine(metric_doc);
if (!replaceInvalidChars(metric_name))
continue;
std::string key{current_status_prefix + metric_name};
writeOutLine(wb, "# HELP", key, metric_doc);
writeOutLine(wb, "# TYPE", key, "gauge");
for (const auto & value: CurrentStatusInfo::values[i])
{
for (const auto & enum_value: CurrentStatusInfo::getAllPossibleValues(static_cast<CurrentStatusInfo::Status>(i)))
{
DB::writeText(key, wb);
DB::writeChar('{', wb);
DB::writeText(key, wb);
DB::writeChar('=', wb);
writeDoubleQuotedString(enum_value.first, wb);
DB::writeText(",name=", wb);
writeDoubleQuotedString(value.first, wb);
DB::writeText("} ", wb);
DB::writeText(value.second == enum_value.second, wb);
DB::writeChar('\n', wb);
}
}
}
}
}
}

View File

@ -27,12 +27,10 @@ private:
const bool send_events;
const bool send_metrics;
const bool send_asynchronous_metrics;
const bool send_status_info;
static inline constexpr auto profile_events_prefix = "ClickHouseProfileEvents_";
static inline constexpr auto current_metrics_prefix = "ClickHouseMetrics_";
static inline constexpr auto asynchronous_metrics_prefix = "ClickHouseAsyncMetrics_";
static inline constexpr auto current_status_prefix = "ClickHouseStatusInfo_";
};
}

View File

@ -674,27 +674,6 @@
</graphite>
-->
<!-- Serve endpoint for Prometheus monitoring. -->
<!--
endpoint - mertics path (relative to root, statring with "/")
port - port to setup server. If not defined or 0 than http_port used
metrics - send data from table system.metrics
events - send data from table system.events
asynchronous_metrics - send data from table system.asynchronous_metrics
status_info - send data from different component from CH, ex: Dictionaries status
-->
<!--
<prometheus>
<endpoint>/metrics</endpoint>
<port>9363</port>
<metrics>true</metrics>
<events>true</events>
<asynchronous_metrics>true</asynchronous_metrics>
<status_info>true</status_info>
</prometheus>
-->
<!-- Query log. Used only for queries with setting log_queries = 1. -->
<query_log>
<!-- What table to insert data. If table is not exist, it will be created.

View File

@ -1,18 +0,0 @@
status before reload
status after reload
NOT_LOADED 0
LOADED 0
FAILED 1
LOADING 0
FAILED_AND_RELOADING 0
LOADED_AND_RELOADING 0
NOT_EXIST 0
status after reload, table exists
NOT_LOADED 0
LOADED 1
FAILED 0
LOADING 0
FAILED_AND_RELOADING 0
LOADED_AND_RELOADING 0
NOT_EXIST 0
status after drop

View File

@ -1,38 +0,0 @@
#!/usr/bin/env bash
# Tags: no-ordinary-database
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
function get_dictionary_status()
{
local name=$1 && shift
$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL_PROMETHEUS" | {
awk -F'[{}=," ]' -vname="$name" '/ClickHouseStatusInfo_DictionaryStatus{/ && $(NF-3) == name { print $4, $NF }'
}
}
$CLICKHOUSE_CLIENT -q "CREATE DICTIONARY dict (key Int, value String) PRIMARY KEY key SOURCE(CLICKHOUSE(TABLE data)) LAYOUT(HASHED()) LIFETIME(0)"
uuid="$($CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.dictionaries WHERE database = '$CLICKHOUSE_DATABASE' AND name = 'dict'")"
echo 'status before reload'
get_dictionary_status "$uuid"
# source table does not exists
# NOTE: when dictionary does not exist it produce BAD_ARGUMENTS error, so using UNKNOWN_TABLE is safe
$CLICKHOUSE_CLIENT -n -q "SYSTEM RELOAD DICTIONARY dict -- { serverError UNKNOWN_TABLE }"
echo 'status after reload'
get_dictionary_status "$uuid"
# create source
$CLICKHOUSE_CLIENT -q "CREATE TABLE data (key Int, value String) Engine=Null"
$CLICKHOUSE_CLIENT -q "SYSTEM RELOAD DICTIONARY dict"
echo 'status after reload, table exists'
get_dictionary_status "$uuid"
# remove dictionary
$CLICKHOUSE_CLIENT -q "DROP DICTIONARY dict"
$CLICKHOUSE_CLIENT -q "DROP TABLE data"
echo 'status after drop'
get_dictionary_status "$uuid"