mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge pull request #26246 from FArthur-cmd/adding-system-warnings-26039
Adding system warnings 26039
This commit is contained in:
commit
d28b143c49
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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();
|
||||
|
@ -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>;
|
||||
|
19
src/Storages/System/StorageSystemWarnings.cpp
Normal file
19
src/Storages/System/StorageSystemWarnings.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
23
src/Storages/System/StorageSystemWarnings.h
Normal file
23
src/Storages/System/StorageSystemWarnings.h
Normal 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;
|
||||
};
|
||||
}
|
@ -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");
|
||||
|
40
tests/queries/0_stateless/01945_system_warnings.expect
Executable file
40
tests/queries/0_stateless/01945_system_warnings.expect
Executable 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
|
Loading…
Reference in New Issue
Block a user