ClickHouse/src/Server/ReplicasStatusHandler.cpp
mateng0915 eae0f9957c Feature: Optimize the replica delay api logic
Description:
============
Currently if in the Whole CK instance has one table is read only status
then the API  /replicas_status will throw error, xxx table is read only

For make this monitor can work in prod env, we can catch the read only
status instead of directly throw error

Solution:
=========
Return other normal table's delay value even if the CK instance has readonly Replicatedxxx table

Please enter the commit message for your changes. Lines starting
2023-01-13 14:44:27 +08:00

129 lines
4.2 KiB
C++

#include <Server/ReplicasStatusHandler.h>
#include <Databases/IDatabase.h>
#include <IO/HTTPCommon.h>
#include <Interpreters/Context.h>
#include <Server/HTTP/HTMLForm.h>
#include <Server/HTTPHandlerFactory.h>
#include <Server/HTTPHandlerRequestFilter.h>
#include <Server/IServer.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Common/typeid_cast.h>
#include <Poco/Net/HTTPRequestHandlerFactory.h>
#include <Poco/Net/HTTPServerRequest.h>
#include <Poco/Net/HTTPServerResponse.h>
namespace DB
{
ReplicasStatusHandler::ReplicasStatusHandler(IServer & server) : WithContext(server.context())
{
}
void ReplicasStatusHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response)
{
try
{
HTMLForm params(getContext()->getSettingsRef(), request);
/// Even if lag is small, output detailed information about the lag.
bool verbose = params.get("verbose", "") == "1";
const MergeTreeSettings & settings = getContext()->getReplicatedMergeTreeSettings();
bool ok = true;
WriteBufferFromOwnString message;
auto databases = DatabaseCatalog::instance().getDatabases();
/// Iterate through all the replicated tables.
for (const auto & db : databases)
{
/// Check if database can contain replicated tables
if (!db.second->canContainMergeTreeTables())
continue;
for (auto iterator = db.second->getTablesIterator(getContext()); iterator->isValid(); iterator->next())
{
const auto & table = iterator->table();
if (!table)
continue;
StorageReplicatedMergeTree * table_replicated = dynamic_cast<StorageReplicatedMergeTree *>(table.get());
if (!table_replicated)
continue;
time_t absolute_delay = 0;
time_t relative_delay = 0;
if(!table_replicated->isTableReadOnly())
{
table_replicated->getReplicaDelays(absolute_delay, relative_delay);
if ((settings.min_absolute_delay_to_close && absolute_delay >= static_cast<time_t>(settings.min_absolute_delay_to_close))
|| (settings.min_relative_delay_to_close && relative_delay >= static_cast<time_t>(settings.min_relative_delay_to_close)))
ok = false;
message << backQuoteIfNeed(db.first) << "." << backQuoteIfNeed(iterator->name())
<< ":\tAbsolute delay: " << absolute_delay << ". Relative delay: " << relative_delay << ".\n";
} else {
message << backQuoteIfNeed(db.first) << "." << backQuoteIfNeed(iterator->name())
<< ":\tis readonly. \n";
}
}
}
const auto & config = getContext()->getConfigRef();
setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", 10));
if (!ok)
{
response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_SERVICE_UNAVAILABLE);
verbose = true;
}
if (verbose)
*response.send() << message.str();
else
{
const char * data = "Ok.\n";
response.sendBuffer(data, strlen(data));
}
}
catch (...)
{
tryLogCurrentException("ReplicasStatusHandler");
try
{
response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR);
if (!response.sent())
{
/// We have not sent anything yet and we don't even know if we need to compress response.
*response.send() << getCurrentExceptionMessage(false) << std::endl;
}
}
catch (...)
{
LOG_ERROR((&Poco::Logger::get("ReplicasStatusHandler")), "Cannot send exception to client");
}
}
}
HTTPRequestHandlerFactoryPtr createReplicasStatusHandlerFactory(IServer & server,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix)
{
auto factory = std::make_shared<HandlingRuleHTTPHandlerFactory<ReplicasStatusHandler>>(server);
factory->addFiltersFromConfig(config, config_prefix);
return factory;
}
}