Merge pull request #26246 from FArthur-cmd/adding-system-warnings-26039

Adding system warnings 26039
This commit is contained in:
alexey-milovidov 2021-07-13 03:44:53 +03:00 committed by GitHub
commit d28b143c49
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 123 additions and 14 deletions

View File

@ -477,17 +477,6 @@ int Server::main(const std::vector<std::string> & /*args*/)
CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision());
CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger());
if (ThreadFuzzer::instance().isEffective())
LOG_WARNING(log, "ThreadFuzzer is enabled. Application will run slowly and unstable.");
#if !defined(NDEBUG) || !defined(__OPTIMIZE__)
LOG_WARNING(log, "Server was built in debug mode. It will work slowly.");
#endif
#if defined(SANITIZER)
LOG_WARNING(log, "Server was built with sanitizer. It will work slowly.");
#endif
/** Context contains all that query execution is dependent:
* settings, available functions, data types, aggregate functions, databases, ...
*/
@ -497,6 +486,18 @@ int Server::main(const std::vector<std::string> & /*args*/)
global_context->makeGlobalContext();
global_context->setApplicationType(Context::ApplicationType::SERVER);
#if !defined(NDEBUG) || !defined(__OPTIMIZE__)
global_context->addWarningMessage("Server was built in debug mode. It will work slowly.");
#endif
if (ThreadFuzzer::instance().isEffective())
global_context->addWarningMessage("ThreadFuzzer is enabled. Application will run slowly and unstable.");
#if defined(SANITIZER)
global_context->addWarningMessage("Server was built with sanitizer. It will work slowly.");
#endif
// Initialize global thread pool. Do it before we fetch configs from zookeeper
// nodes (`from_zk`), because ZooKeeper interface uses the pool. We will
// ignore `max_thread_pool_size` in configs we fetch from ZK, but oh well.
@ -552,8 +553,10 @@ int Server::main(const std::vector<std::string> & /*args*/)
if (ptrace(PTRACE_TRACEME, 0, nullptr, nullptr) == -1)
{
/// Program is run under debugger. Modification of it's binary image is ok for breakpoints.
LOG_WARNING(log, "Server is run under debugger and its binary image is modified (most likely with breakpoints).",
calculated_binary_hash);
global_context->addWarningMessage(
fmt::format("Server is run under debugger and its binary image is modified (most likely with breakpoints).",
calculated_binary_hash)
);
}
else
{
@ -636,7 +639,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
}
else
{
LOG_WARNING(log, message);
global_context->addWarningMessage(message);
}
}

View File

@ -386,6 +386,7 @@ struct ContextSharedPart
ActionLocksManagerPtr action_locks_manager; /// Set of storages' action lockers
std::unique_ptr<SystemLogs> system_logs; /// Used to log queries and operations on parts
std::optional<StorageS3Settings> storage_s3_settings; /// Settings of S3 storage
std::vector<String> warnings_logs; /// Store warning messages
RemoteHostFilter remote_host_filter; /// Allowed URL from config.xml
@ -514,6 +515,12 @@ struct ContextSharedPart
trace_collector.emplace(std::move(trace_log));
}
void addWarningMessage(const String& message)
{
log->warning(message);
warnings_logs.push_back(message);
}
};
@ -635,6 +642,12 @@ String Context::getDictionariesLibPath() const
return shared->dictionaries_lib_path;
}
std::vector<String> Context::getWarnings() const
{
auto lock = getLock();
return shared->warnings_logs;
}
VolumePtr Context::getTemporaryVolume() const
{
auto lock = getLock();
@ -706,6 +719,12 @@ void Context::setDictionariesLibPath(const String & path)
shared->dictionaries_lib_path = path;
}
void Context::addWarningMessage(const String & msg)
{
auto lock = getLock();
shared->addWarningMessage(msg);
}
void Context::setConfig(const ConfigurationPtr & config)
{
auto lock = getLock();

View File

@ -318,6 +318,7 @@ public:
String getFlagsPath() const;
String getUserFilesPath() const;
String getDictionariesLibPath() const;
std::vector<String> getWarnings() const;
VolumePtr getTemporaryVolume() const;
@ -326,6 +327,8 @@ public:
void setUserFilesPath(const String & path);
void setDictionariesLibPath(const String & path);
void addWarningMessage(const String & msg);
VolumePtr setTemporaryStorage(const String & path, const String & policy_name = "");
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;

View File

@ -0,0 +1,19 @@
#include <Interpreters/Context.h>
#include <Storages/System/StorageSystemWarnings.h>
namespace DB
{
NamesAndTypesList StorageSystemWarnings::getNamesAndTypes()
{
return {
{"message", std::make_shared<DataTypeString>()},
};
}
void StorageSystemWarnings::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
for (const auto & warning : context->getWarnings())
res_columns[0]->insert(warning);
}
}

View File

@ -0,0 +1,23 @@
#pragma once
#include <Storages/System/IStorageSystemOneBlock.h>
namespace DB
{
class Context;
class StorageSystemWarnings final : public shared_ptr_helper<StorageSystemWarnings>,
public IStorageSystemOneBlock<StorageSystemWarnings> {
public:
std::string getName() const override { return "SystemWarnings"; }
static NamesAndTypesList getNamesAndTypes();
protected:
friend struct shared_ptr_helper<StorageSystemWarnings>;
using IStorageSystemOneBlock::IStorageSystemOneBlock;
void fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const override;
};
}

View File

@ -43,6 +43,7 @@
#include <Storages/System/StorageSystemZooKeeper.h>
#include <Storages/System/StorageSystemContributors.h>
#include <Storages/System/StorageSystemErrors.h>
#include <Storages/System/StorageSystemWarnings.h>
#include <Storages/System/StorageSystemDDLWorkerQueue.h>
#if !defined(ARCADIA_BUILD)
@ -116,6 +117,7 @@ void attachSystemTablesLocal(IDatabase & system_database)
attach<StorageSystemUserDirectories>(system_database, "user_directories");
attach<StorageSystemPrivileges>(system_database, "privileges");
attach<StorageSystemErrors>(system_database, "errors");
attach<StorageSystemWarnings>(system_database, "warnings");
attach<StorageSystemDataSkippingIndices>(system_database, "data_skipping_indices");
#if !defined(ARCADIA_BUILD)
attach<StorageSystemLicenses>(system_database, "licenses");

View File

@ -0,0 +1,40 @@
#!/usr/bin/expect -f
# This is a test for system.warnings. Testing in interactive mode is necessary,
# as we want to see certain warnings from client
log_user 0
set timeout 60
match_max 100000
# A default timeout action is to do nothing, change it to fail
expect_after {
timeout {
exit 1
}
}
set basedir [file dirname $argv0]
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion"
expect ":) "
#find out BUILD TYPE
send -- "SELECT value FROM system.build_options WHERE name='BUILD_TYPE'\r"
expect {
"Debug" {
# Check debug message in system.warnings
send -- "SELECT message FROM system.warnings WHERE message='Server was built in debug mode. It will work slowly.'\r"
expect "Server was built in debug mode. It will work slowly."
expect ":) "
}
"RelWithDebInfo" {
# Check empty to find out existence
send -- "SELECT message FROM system.warnings WHERE 0=1\r"
expect "Ok."
expect ":) "
}
}
# Finish test
send -- "\4"
expect eof