Merge branch 'master' into parallel-log-appending

This commit is contained in:
Antonio Andelic 2022-12-01 14:44:36 +00:00
commit 7cd76df918
136 changed files with 2796 additions and 1252 deletions

View File

@ -295,6 +295,9 @@ if not args.use_existing_tables:
reportStageEnd("create")
# Let's sync the data to avoid writeback affects performance
os.system("sync")
# By default, test all queries.
queries_to_run = range(0, len(test_queries))

View File

@ -17,7 +17,7 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \
python3-pip \
shellcheck \
yamllint \
&& pip3 install black==22.8.0 boto3 codespell==2.2.1 dohq-artifactory PyGithub unidiff pylint==2.6.2 \
&& pip3 install black==22.8.0 boto3 codespell==2.2.1 dohq-artifactory mypy PyGithub unidiff pylint==2.6.2 \
&& apt-get clean \
&& rm -rf /root/.cache/pip

View File

@ -11,17 +11,19 @@ def process_result(result_folder):
description = ""
test_results = []
checks = (
("header duplicates", "duplicate_output.txt"),
("shellcheck", "shellcheck_output.txt"),
("style", "style_output.txt"),
("black", "black_output.txt"),
("typos", "typos_output.txt"),
("whitespaces", "whitespaces_output.txt"),
("workflows", "workflows_output.txt"),
("doc typos", "doc_spell_output.txt"),
"duplicate includes",
"shellcheck",
"style",
"black",
"mypy",
"typos",
"whitespaces",
"workflows",
"docs spelling",
)
for name, out_file in checks:
for name in checks:
out_file = name.replace(" ", "_") + "_output.txt"
full_path = os.path.join(result_folder, out_file)
if not os.path.exists(full_path):
logging.info("No %s check log on path %s", name, full_path)

View File

@ -4,15 +4,17 @@
cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_output/check_status.tsv
echo "Check duplicates" | ts
./check-duplicate-includes.sh |& tee /test_output/duplicate_output.txt
./check-duplicate-includes.sh |& tee /test_output/duplicate_includes_output.txt
echo "Check style" | ts
./check-style -n |& tee /test_output/style_output.txt
echo "Check python formatting with black" | ts
./check-black -n |& tee /test_output/black_output.txt
echo "Check python type hinting with mypy" | ts
./check-mypy -n |& tee /test_output/mypy_output.txt
echo "Check typos" | ts
./check-typos |& tee /test_output/typos_output.txt
echo "Check docs spelling" | ts
./check-doc-aspell |& tee /test_output/doc_spell_output.txt
./check-doc-aspell |& tee /test_output/docs_spelling_output.txt
echo "Check whitespaces" | ts
./check-whitespaces -n |& tee /test_output/whitespaces_output.txt
echo "Check workflows" | ts

View File

@ -19,7 +19,7 @@ then
# Will make a repository with website content as the only commit.
git init
git remote add origin "${GIT_PROD_URI}"
git config user.email "robot-clickhouse@clickhouse.com"
git config user.email "robot-clickhouse@users.noreply.github.com"
git config user.name "robot-clickhouse"
# Add files.

View File

@ -13,7 +13,6 @@ clickhouse_embed_binaries(
set(CLICKHOUSE_KEEPER_SOURCES
Keeper.cpp
TinyContext.cpp
)
set (CLICKHOUSE_KEEPER_LINK
@ -49,6 +48,8 @@ if (BUILD_STANDALONE_KEEPER)
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateMachine.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateManager.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStorage.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperAsynchronousMetrics.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/TinyContext.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/pathUtils.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/SessionExpiryQueue.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/SummingStateMachine.cpp
@ -64,7 +65,18 @@ if (BUILD_STANDALONE_KEEPER)
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/KeeperTCPHandler.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/TCPServer.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/NotFoundHandler.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/ProtocolServerAdapter.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/PrometheusRequestHandler.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/PrometheusMetricsWriter.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTPRequestHandlerFactoryMain.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/HTTPServer.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/ReadHeaders.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/HTTPServerConnection.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/HTTPServerRequest.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/HTTPServerResponse.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/HTTPServerConnectionFactory.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CachedCompressedReadBuffer.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CheckingCompressedReadBuffer.cpp
@ -96,9 +108,7 @@ if (BUILD_STANDALONE_KEEPER)
${CMAKE_CURRENT_BINARY_DIR}/../../src/Daemon/GitHash.generated.cpp
Keeper.cpp
TinyContext.cpp
clickhouse-keeper.cpp
)
clickhouse_add_executable(clickhouse-keeper ${CLICKHOUSE_KEEPER_STANDALONE_SOURCES})

View File

@ -6,7 +6,6 @@
#include <Interpreters/DNSCacheUpdater.h>
#include <Coordination/Defines.h>
#include <Common/Config/ConfigReloader.h>
#include <Server/TCPServer.h>
#include <filesystem>
#include <IO/UseSSL.h>
#include <Core/ServerUUID.h>
@ -22,8 +21,15 @@
#include <Poco/Environment.h>
#include <sys/stat.h>
#include <pwd.h>
#include <Coordination/FourLetterCommand.h>
#include <Coordination/FourLetterCommand.h>
#include <Coordination/KeeperAsynchronousMetrics.h>
#include <Server/HTTP/HTTPServer.h>
#include <Server/TCPServer.h>
#include <Server/HTTPHandlerFactory.h>
#include "Core/Defines.h"
#include "config.h"
#include "config_version.h"
@ -52,6 +58,16 @@ int mainEntryClickHouseKeeper(int argc, char ** argv)
}
}
#ifdef KEEPER_STANDALONE_BUILD
// Weak symbols don't work correctly on Darwin
// so we have a stub implementation to avoid linker errors
void collectCrashLog(
Int32, UInt64, const String &, const StackTrace &)
{}
#endif
namespace DB
{
@ -261,6 +277,60 @@ void Keeper::defineOptions(Poco::Util::OptionSet & options)
BaseDaemon::defineOptions(options);
}
struct Keeper::KeeperHTTPContext : public IHTTPContext
{
explicit KeeperHTTPContext(TinyContextPtr context_)
: context(std::move(context_))
{}
uint64_t getMaxHstsAge() const override
{
return context->getConfigRef().getUInt64("keeper_server.hsts_max_age", 0);
}
uint64_t getMaxUriSize() const override
{
return context->getConfigRef().getUInt64("keeper_server.http_max_uri_size", 1048576);
}
uint64_t getMaxFields() const override
{
return context->getConfigRef().getUInt64("keeper_server.http_max_fields", 1000000);
}
uint64_t getMaxFieldNameSize() const override
{
return context->getConfigRef().getUInt64("keeper_server.http_max_field_name_size", 1048576);
}
uint64_t getMaxFieldValueSize() const override
{
return context->getConfigRef().getUInt64("keeper_server.http_max_field_value_size", 1048576);
}
uint64_t getMaxChunkSize() const override
{
return context->getConfigRef().getUInt64("keeper_server.http_max_chunk_size", 100_GiB);
}
Poco::Timespan getReceiveTimeout() const override
{
return context->getConfigRef().getUInt64("keeper_server.http_receive_timeout", DEFAULT_HTTP_READ_BUFFER_TIMEOUT);
}
Poco::Timespan getSendTimeout() const override
{
return context->getConfigRef().getUInt64("keeper_server.http_send_timeout", DEFAULT_HTTP_READ_BUFFER_TIMEOUT);
}
TinyContextPtr context;
};
HTTPContextPtr Keeper::httpContext()
{
return std::make_shared<KeeperHTTPContext>(tiny_context);
}
int Keeper::main(const std::vector<std::string> & /*args*/)
try
{
@ -335,6 +405,25 @@ try
DNSResolver::instance().setDisableCacheFlag();
Poco::ThreadPool server_pool(3, config().getUInt("max_connections", 1024));
std::mutex servers_lock;
auto servers = std::make_shared<std::vector<ProtocolServerAdapter>>();
tiny_context = std::make_shared<TinyContext>();
/// This object will periodically calculate some metrics.
KeeperAsynchronousMetrics async_metrics(
tiny_context,
config().getUInt("asynchronous_metrics_update_period_s", 1),
[&]() -> std::vector<ProtocolServerMetrics>
{
std::vector<ProtocolServerMetrics> metrics;
std::lock_guard lock(servers_lock);
metrics.reserve(servers->size());
for (const auto & server : *servers)
metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()});
return metrics;
}
);
std::vector<std::string> listen_hosts = DB::getMultipleValuesFromConfig(config(), "", "listen_host");
@ -346,15 +435,13 @@ try
listen_try = true;
}
auto servers = std::make_shared<std::vector<ProtocolServerAdapter>>();
/// Initialize keeper RAFT. Do nothing if no keeper_server in config.
tiny_context.initializeKeeperDispatcher(/* start_async = */ true);
FourLetterCommandFactory::registerCommands(*tiny_context.getKeeperDispatcher());
tiny_context->initializeKeeperDispatcher(/* start_async = */ true);
FourLetterCommandFactory::registerCommands(*tiny_context->getKeeperDispatcher());
auto config_getter = [this] () -> const Poco::Util::AbstractConfiguration &
{
return tiny_context.getConfigRef();
return tiny_context->getConfigRef();
};
for (const auto & listen_host : listen_hosts)
@ -373,7 +460,7 @@ try
"Keeper (tcp): " + address.toString(),
std::make_unique<TCPServer>(
new KeeperTCPHandlerFactory(
config_getter, tiny_context.getKeeperDispatcher(),
config_getter, tiny_context->getKeeperDispatcher(),
config().getUInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC),
config().getUInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC), false), server_pool, socket));
});
@ -392,7 +479,7 @@ try
"Keeper with secure protocol (tcp_secure): " + address.toString(),
std::make_unique<TCPServer>(
new KeeperTCPHandlerFactory(
config_getter, tiny_context.getKeeperDispatcher(),
config_getter, tiny_context->getKeeperDispatcher(),
config().getUInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC),
config().getUInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC), true), server_pool, socket));
#else
@ -401,6 +488,29 @@ try
ErrorCodes::SUPPORT_IS_DISABLED};
#endif
});
const auto & config = config_getter();
Poco::Timespan keep_alive_timeout(config.getUInt("keep_alive_timeout", 10), 0);
Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams;
http_params->setTimeout(DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC);
http_params->setKeepAliveTimeout(keep_alive_timeout);
/// Prometheus (if defined and not setup yet with http_port)
port_name = "prometheus.port";
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
{
Poco::Net::ServerSocket socket;
auto address = socketBindListen(socket, listen_host, port);
auto http_context = httpContext();
socket.setReceiveTimeout(http_context->getReceiveTimeout());
socket.setSendTimeout(http_context->getSendTimeout());
servers->emplace_back(
listen_host,
port_name,
"Prometheus: http://" + address.toString(),
std::make_unique<HTTPServer>(
std::move(http_context), createPrometheusMainHandlerFactory(*this, config_getter(), async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params));
});
}
for (auto & server : *servers)
@ -409,6 +519,8 @@ try
LOG_INFO(log, "Listening for {}", server.getDescription());
}
async_metrics.start();
zkutil::EventPtr unused_event = std::make_shared<Poco::Event>();
zkutil::ZooKeeperNodeCache unused_cache([] { return nullptr; });
/// ConfigReloader have to strict parameters which are redundant in our case
@ -421,7 +533,7 @@ try
[&](ConfigurationPtr config, bool /* initial_loading */)
{
if (config->has("keeper_server"))
tiny_context.updateKeeperConfiguration(*config);
tiny_context->updateKeeperConfiguration(*config);
},
/* already_loaded = */ false); /// Reload it right now (initial loading)
@ -429,6 +541,8 @@ try
LOG_INFO(log, "Shutting down.");
main_config_reloader.reset();
async_metrics.stop();
LOG_DEBUG(log, "Waiting for current connections to Keeper to finish.");
size_t current_connections = 0;
for (auto & server : *servers)
@ -450,7 +564,7 @@ try
else
LOG_INFO(log, "Closed connections to Keeper.");
tiny_context.shutdownKeeperDispatcher();
tiny_context->shutdownKeeperDispatcher();
/// Wait server pool to avoid use-after-free of destroyed context in the handlers
server_pool.joinAll();

View File

@ -1,8 +1,9 @@
#pragma once
#include <Server/IServer.h>
#include <Server/HTTP/HTTPContext.h>
#include <Daemon/BaseDaemon.h>
#include "TinyContext.h"
#include <Coordination/TinyContext.h>
namespace Poco
{
@ -15,29 +16,40 @@ namespace Poco
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
/// standalone clickhouse-keeper server (replacement for ZooKeeper). Uses the same
/// config as clickhouse-server. Serves requests on TCP ports with or without
/// SSL using ZooKeeper protocol.
class Keeper : public BaseDaemon
class Keeper : public BaseDaemon, public IServer
{
public:
using ServerApplication::run;
Poco::Util::LayeredConfiguration & config() const
Poco::Util::LayeredConfiguration & config() const override
{
return BaseDaemon::config();
}
Poco::Logger & logger() const
Poco::Logger & logger() const override
{
return BaseDaemon::logger();
}
bool isCancelled() const
bool isCancelled() const override
{
return BaseDaemon::isCancelled();
}
/// Returns global application's context.
ContextMutablePtr context() const override
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot fetch context for Keeper");
}
void defineOptions(Poco::Util::OptionSet & _options) override;
protected:
@ -56,7 +68,10 @@ protected:
std::string getDefaultConfigFileName() const override;
private:
TinyContext tiny_context;
TinyContextPtr tiny_context;
struct KeeperHTTPContext;
HTTPContextPtr httpContext();
Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) const;

View File

@ -207,7 +207,7 @@ void LocalServer::tryInitPath()
global_context->setPath(path);
global_context->setTemporaryStorage(path + "tmp", "", 0);
global_context->setTemporaryStoragePath(path + "tmp/", 0);
global_context->setFlagsPath(path + "flags");
global_context->setUserFilesPath(""); // user's files are everywhere

View File

@ -1,6 +1,6 @@
#include "MetricsTransmitter.h"
#include <Interpreters/AsynchronousMetrics.h>
#include <Common/AsynchronousMetrics.h>
#include <Common/CurrentMetrics.h>
#include <Common/Exception.h>

View File

@ -46,7 +46,7 @@
#include <IO/ReadBufferFromFile.h>
#include <IO/IOThreadPool.h>
#include <IO/UseSSL.h>
#include <Interpreters/AsynchronousMetrics.h>
#include <Interpreters/ServerAsynchronousMetrics.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/DNSCacheUpdater.h>
#include <Interpreters/DatabaseCatalog.h>
@ -203,46 +203,6 @@ int mainEntryClickHouseServer(int argc, char ** argv)
namespace
{
void setupTmpPath(Poco::Logger * log, const std::string & path)
try
{
LOG_DEBUG(log, "Setting up {} to store temporary data in it", path);
fs::create_directories(path);
/// Clearing old temporary files.
fs::directory_iterator dir_end;
size_t unknown_files = 0;
for (fs::directory_iterator it(path); it != dir_end; ++it)
{
if (it->is_regular_file() && startsWith(it->path().filename(), "tmp"))
{
LOG_DEBUG(log, "Removing old temporary file {}", it->path().string());
fs::remove(it->path());
}
else
{
unknown_files++;
if (unknown_files < 100)
LOG_DEBUG(log, "Found unknown {} {} in temporary path",
it->is_regular_file() ? "file" : (it->is_directory() ? "directory" : "element"),
it->path().string());
}
}
if (unknown_files)
LOG_DEBUG(log, "Found {} unknown files in temporary path", unknown_files);
}
catch (...)
{
DB::tryLogCurrentException(
log,
fmt::format(
"Caught exception while setup temporary path: {}. It is ok to skip this exception as cleaning old temporary files is not "
"necessary",
path));
}
size_t waitServersToFinish(std::vector<DB::ProtocolServerAdapter> & servers, size_t seconds_to_wait)
{
const size_t sleep_max_ms = 1000 * seconds_to_wait;
@ -803,7 +763,7 @@ try
std::vector<ProtocolServerAdapter> servers;
std::vector<ProtocolServerAdapter> servers_to_start_before_tables;
/// This object will periodically calculate some metrics.
AsynchronousMetrics async_metrics(
ServerAsynchronousMetrics async_metrics(
global_context,
config().getUInt("asynchronous_metrics_update_period_s", 1),
config().getUInt("asynchronous_heavy_metrics_update_period_s", 120),
@ -1037,13 +997,21 @@ try
LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::instance().getTimeZone());
/// Storage with temporary data for processing of heavy queries.
if (auto temporary_policy = config().getString("tmp_policy", ""); !temporary_policy.empty())
{
size_t max_size = config().getUInt64("max_temporary_data_on_disk_size", 0);
global_context->setTemporaryStoragePolicy(temporary_policy, max_size);
}
else if (auto temporary_cache = config().getString("tmp_cache", ""); !temporary_cache.empty())
{
size_t max_size = config().getUInt64("max_temporary_data_on_disk_size", 0);
global_context->setTemporaryStorageInCache(temporary_cache, max_size);
}
else
{
std::string temporary_path = config().getString("tmp_path", path / "tmp/");
std::string temporary_policy = config().getString("tmp_policy", "");
size_t max_size = config().getUInt64("max_temporary_data_on_disk_size", 0);
const VolumePtr & volume = global_context->setTemporaryStorage(temporary_path, temporary_policy, max_size);
for (const DiskPtr & disk : volume->getDisks())
setupTmpPath(log, disk->getPath());
global_context->setTemporaryStoragePath(temporary_path, max_size);
}
/** Directory with 'flags': files indicating temporary settings for the server set by system administrator.
@ -1442,7 +1410,7 @@ try
}
catch (...)
{
tryLogCurrentException(log);
tryLogCurrentException(log, "Caught exception while setting up access control.");
throw;
}
@ -1947,15 +1915,15 @@ std::unique_ptr<TCPProtocolStackFactory> Server::buildProtocolStackFromConfig(
return TCPServerConnectionFactory::Ptr(new PostgreSQLHandlerFactory(*this));
if (type == "http")
return TCPServerConnectionFactory::Ptr(
new HTTPServerConnectionFactory(context(), http_params, createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory"))
new HTTPServerConnectionFactory(httpContext(), http_params, createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory"))
);
if (type == "prometheus")
return TCPServerConnectionFactory::Ptr(
new HTTPServerConnectionFactory(context(), http_params, createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"))
new HTTPServerConnectionFactory(httpContext(), http_params, createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"))
);
if (type == "interserver")
return TCPServerConnectionFactory::Ptr(
new HTTPServerConnectionFactory(context(), http_params, createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory"))
new HTTPServerConnectionFactory(httpContext(), http_params, createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory"))
);
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol configuration error, unknown protocol name '{}'", type);
@ -1996,6 +1964,11 @@ std::unique_ptr<TCPProtocolStackFactory> Server::buildProtocolStackFromConfig(
return stack;
}
HTTPContextPtr Server::httpContext() const
{
return std::make_shared<HTTPContext>(context());
}
void Server::createServers(
Poco::Util::AbstractConfiguration & config,
const Strings & listen_hosts,
@ -2078,7 +2051,7 @@ void Server::createServers(
port_name,
"http://" + address.toString(),
std::make_unique<HTTPServer>(
context(), createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params));
httpContext(), createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params));
});
/// HTTPS
@ -2095,7 +2068,7 @@ void Server::createServers(
port_name,
"https://" + address.toString(),
std::make_unique<HTTPServer>(
context(), createHandlerFactory(*this, config, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params));
httpContext(), createHandlerFactory(*this, config, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params));
#else
UNUSED(port);
throw Exception{"HTTPS protocol is disabled because Poco library was built without NetSSL support.",
@ -2220,7 +2193,7 @@ void Server::createServers(
port_name,
"Prometheus: http://" + address.toString(),
std::make_unique<HTTPServer>(
context(), createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params));
httpContext(), createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params));
});
}
@ -2240,7 +2213,7 @@ void Server::createServers(
port_name,
"replica communication (interserver): http://" + address.toString(),
std::make_unique<HTTPServer>(
context(),
httpContext(),
createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory"),
server_pool,
socket,
@ -2260,7 +2233,7 @@ void Server::createServers(
port_name,
"secure replica communication (interserver): https://" + address.toString(),
std::make_unique<HTTPServer>(
context(),
httpContext(),
createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPSHandler-factory"),
server_pool,
socket,

View File

@ -3,6 +3,7 @@
#include <Server/IServer.h>
#include <Daemon/BaseDaemon.h>
#include "Server/HTTP/HTTPContext.h"
#include <Server/TCPProtocolStackFactory.h>
#include <Poco/Net/HTTPServerParams.h>
@ -72,6 +73,8 @@ private:
/// Updated/recent config, to compare http_handlers
ConfigurationPtr latest_config;
HTTPContextPtr httpContext() const;
Poco::Net::SocketAddress socketBindListen(
const Poco::Util::AbstractConfiguration & config,
Poco::Net::ServerSocket & socket,

View File

@ -166,7 +166,8 @@ void BackupWriterS3::copyObjectImpl(
auto outcome = client->CopyObject(request);
if (!outcome.IsSuccess() && outcome.GetError().GetExceptionName() == "EntityTooLarge")
if (!outcome.IsSuccess() && (outcome.GetError().GetExceptionName() == "EntityTooLarge"
|| outcome.GetError().GetExceptionName() == "InvalidRequest"))
{ // Can't come here with MinIO, MinIO allows single part upload for large objects.
copyObjectMultipartImpl(src_bucket, src_key, dst_bucket, dst_key, head, metadata);
return;

View File

@ -236,7 +236,7 @@ int IBridge::main(const std::vector<std::string> & /*args*/)
SensitiveDataMasker::setInstance(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
auto server = HTTPServer(
context,
std::make_shared<HTTPContext>(context),
getHandlerFactoryPtr(context),
server_pool,
socket,

View File

@ -1,28 +1,16 @@
#include <Interpreters/Aggregator.h>
#include <Interpreters/AsynchronousMetrics.h>
#include <Interpreters/AsynchronousMetricLog.h>
#include <Interpreters/JIT/CompiledExpressionCache.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/Context.h>
#include <Coordination/Keeper4LWInfo.h>
#include <Coordination/KeeperDispatcher.h>
#include <Common/AsynchronousMetrics.h>
#include <Common/Exception.h>
#include <Common/setThreadName.h>
#include <Common/CurrentMetrics.h>
#include <Common/typeid_cast.h>
#include <Common/filesystemHelpers.h>
#include <Interpreters/Cache/FileCacheFactory.h>
#include <Common/getCurrentProcessFDCount.h>
#include <Common/getMaxFileDescriptorCount.h>
#include <Interpreters/Cache/FileCache.h>
#include <Storages/MarkCache.h>
#include <Storages/StorageMergeTree.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/MergeTree/MergeTreeMetadataCache.h>
#include <Server/ProtocolServerAdapter.h>
#include <IO/UncompressedCache.h>
#include <IO/MMappedFileCache.h>
#include <IO/ReadHelpers.h>
#include <Databases/IDatabase.h>
#include <base/errnoToString.h>
#include <chrono>
@ -68,15 +56,11 @@ static std::unique_ptr<ReadBufferFromFilePRead> openFileIfExists(const std::stri
AsynchronousMetrics::AsynchronousMetrics(
ContextPtr global_context_,
int update_period_seconds,
int heavy_metrics_update_period_seconds,
const ProtocolServerMetricsFunc & protocol_server_metrics_func_)
: WithContext(global_context_)
, update_period(update_period_seconds)
, heavy_metric_update_period(heavy_metrics_update_period_seconds)
, protocol_server_metrics_func(protocol_server_metrics_func_)
: update_period(update_period_seconds)
, log(&Poco::Logger::get("AsynchronousMetrics"))
, protocol_server_metrics_func(protocol_server_metrics_func_)
{
#if defined(OS_LINUX)
openFileIfExists("/proc/meminfo", meminfo);
@ -360,22 +344,6 @@ void AsynchronousMetrics::run()
}
}
template <typename Max, typename T>
static void calculateMax(Max & max, T x)
{
if (Max(x) > max)
max = x;
}
template <typename Max, typename Sum, typename T>
static void calculateMaxAndSum(Max & max, Sum & sum, T x)
{
sum += x;
if (Max(x) > max)
max = x;
}
#if USE_JEMALLOC
uint64_t updateJemallocEpoch()
{
@ -575,91 +543,6 @@ void AsynchronousMetrics::update(TimePoint update_time)
"The difference in time the thread for calculation of the asynchronous metrics was scheduled to wake up and the time it was in fact, woken up."
" A proxy-indicator of overall system latency and responsiveness." };
if (auto mark_cache = getContext()->getMarkCache())
{
new_values["MarkCacheBytes"] = { mark_cache->weight(), "Total size of mark cache in bytes" };
new_values["MarkCacheFiles"] = { mark_cache->count(), "Total number of mark files cached in the mark cache" };
}
if (auto uncompressed_cache = getContext()->getUncompressedCache())
{
new_values["UncompressedCacheBytes"] = { uncompressed_cache->weight(),
"Total size of uncompressed cache in bytes. Uncompressed cache does not usually improve the performance and should be mostly avoided." };
new_values["UncompressedCacheCells"] = { uncompressed_cache->count(),
"Total number of entries in the uncompressed cache. Each entry represents a decompressed block of data. Uncompressed cache does not usually improve performance and should be mostly avoided." };
}
if (auto index_mark_cache = getContext()->getIndexMarkCache())
{
new_values["IndexMarkCacheBytes"] = { index_mark_cache->weight(), "Total size of mark cache for secondary indices in bytes." };
new_values["IndexMarkCacheFiles"] = { index_mark_cache->count(), "Total number of mark files cached in the mark cache for secondary indices." };
}
if (auto index_uncompressed_cache = getContext()->getIndexUncompressedCache())
{
new_values["IndexUncompressedCacheBytes"] = { index_uncompressed_cache->weight(),
"Total size of uncompressed cache in bytes for secondary indices. Uncompressed cache does not usually improve the performance and should be mostly avoided." };
new_values["IndexUncompressedCacheCells"] = { index_uncompressed_cache->count(),
"Total number of entries in the uncompressed cache for secondary indices. Each entry represents a decompressed block of data. Uncompressed cache does not usually improve performance and should be mostly avoided." };
}
if (auto mmap_cache = getContext()->getMMappedFileCache())
{
new_values["MMapCacheCells"] = { mmap_cache->count(),
"The number of files opened with `mmap` (mapped in memory)."
" This is used for queries with the setting `local_filesystem_read_method` set to `mmap`."
" The files opened with `mmap` are kept in the cache to avoid costly TLB flushes."};
}
{
auto caches = FileCacheFactory::instance().getAll();
size_t total_bytes = 0;
size_t total_files = 0;
for (const auto & [_, cache_data] : caches)
{
total_bytes += cache_data->cache->getUsedCacheSize();
total_files += cache_data->cache->getFileSegmentsNum();
}
new_values["FilesystemCacheBytes"] = { total_bytes,
"Total bytes in the `cache` virtual filesystem. This cache is hold on disk." };
new_values["FilesystemCacheFiles"] = { total_files,
"Total number of cached file segments in the `cache` virtual filesystem. This cache is hold on disk." };
}
#if USE_ROCKSDB
if (auto metadata_cache = getContext()->tryGetMergeTreeMetadataCache())
{
new_values["MergeTreeMetadataCacheSize"] = { metadata_cache->getEstimateNumKeys(),
"The size of the metadata cache for tables. This cache is experimental and not used in production." };
}
#endif
#if USE_EMBEDDED_COMPILER
if (auto * compiled_expression_cache = CompiledExpressionCacheFactory::instance().tryGetCache())
{
new_values["CompiledExpressionCacheBytes"] = { compiled_expression_cache->weight(),
"Total bytes used for the cache of JIT-compiled code." };
new_values["CompiledExpressionCacheCount"] = { compiled_expression_cache->count(),
"Total entries in the cache of JIT-compiled code." };
}
#endif
new_values["Uptime"] = { getContext()->getUptimeSeconds(),
"The server uptime in seconds. It includes the time spent for server initialization before accepting connections." };
if (const auto stats = getHashTablesCacheStatistics())
{
new_values["HashTableStatsCacheEntries"] = { stats->entries,
"The number of entries in the cache of hash table sizes."
" The cache for hash table sizes is used for predictive optimization of GROUP BY." };
new_values["HashTableStatsCacheHits"] = { stats->hits,
"The number of times the prediction of a hash table size was correct." };
new_values["HashTableStatsCacheMisses"] = { stats->misses,
"The number of times the prediction of a hash table size was incorrect." };
}
#if defined(OS_LINUX) || defined(OS_FREEBSD)
MemoryStatisticsOS::Data memory_statistics_data = memory_stat.get();
#endif
@ -1519,165 +1402,7 @@ void AsynchronousMetrics::update(TimePoint update_time)
}
#endif
/// Free space in filesystems at data path and logs path.
{
auto stat = getStatVFS(getContext()->getPath());
new_values["FilesystemMainPathTotalBytes"] = { stat.f_blocks * stat.f_frsize,
"The size of the volume where the main ClickHouse path is mounted, in bytes." };
new_values["FilesystemMainPathAvailableBytes"] = { stat.f_bavail * stat.f_frsize,
"Available bytes on the volume where the main ClickHouse path is mounted." };
new_values["FilesystemMainPathUsedBytes"] = { (stat.f_blocks - stat.f_bavail) * stat.f_frsize,
"Used bytes on the volume where the main ClickHouse path is mounted." };
new_values["FilesystemMainPathTotalINodes"] = { stat.f_files,
"The total number of inodes on the volume where the main ClickHouse path is mounted. If it is less than 25 million, it indicates a misconfiguration." };
new_values["FilesystemMainPathAvailableINodes"] = { stat.f_favail,
"The number of available inodes on the volume where the main ClickHouse path is mounted. If it is close to zero, it indicates a misconfiguration, and you will get 'no space left on device' even when the disk is not full." };
new_values["FilesystemMainPathUsedINodes"] = { stat.f_files - stat.f_favail,
"The number of used inodes on the volume where the main ClickHouse path is mounted. This value mostly corresponds to the number of files." };
}
{
/// Current working directory of the server is the directory with logs.
auto stat = getStatVFS(".");
new_values["FilesystemLogsPathTotalBytes"] = { stat.f_blocks * stat.f_frsize,
"The size of the volume where ClickHouse logs path is mounted, in bytes. It's recommended to have at least 10 GB for logs." };
new_values["FilesystemLogsPathAvailableBytes"] = { stat.f_bavail * stat.f_frsize,
"Available bytes on the volume where ClickHouse logs path is mounted. If this value approaches zero, you should tune the log rotation in the configuration file." };
new_values["FilesystemLogsPathUsedBytes"] = { (stat.f_blocks - stat.f_bavail) * stat.f_frsize,
"Used bytes on the volume where ClickHouse logs path is mounted." };
new_values["FilesystemLogsPathTotalINodes"] = { stat.f_files,
"The total number of inodes on the volume where ClickHouse logs path is mounted." };
new_values["FilesystemLogsPathAvailableINodes"] = { stat.f_favail,
"The number of available inodes on the volume where ClickHouse logs path is mounted." };
new_values["FilesystemLogsPathUsedINodes"] = { stat.f_files - stat.f_favail,
"The number of used inodes on the volume where ClickHouse logs path is mounted." };
}
/// Free and total space on every configured disk.
{
DisksMap disks_map = getContext()->getDisksMap();
for (const auto & [name, disk] : disks_map)
{
auto total = disk->getTotalSpace();
/// Some disks don't support information about the space.
if (!total)
continue;
auto available = disk->getAvailableSpace();
auto unreserved = disk->getUnreservedSpace();
new_values[fmt::format("DiskTotal_{}", name)] = { total,
"The total size in bytes of the disk (virtual filesystem). Remote filesystems can show a large value like 16 EiB." };
new_values[fmt::format("DiskUsed_{}", name)] = { total - available,
"Used bytes on the disk (virtual filesystem). Remote filesystems not always provide this information." };
new_values[fmt::format("DiskAvailable_{}", name)] = { available,
"Available bytes on the disk (virtual filesystem). Remote filesystems can show a large value like 16 EiB." };
new_values[fmt::format("DiskUnreserved_{}", name)] = { unreserved,
"Available bytes on the disk (virtual filesystem) without the reservations for merges, fetches, and moves. Remote filesystems can show a large value like 16 EiB." };
}
}
{
auto databases = DatabaseCatalog::instance().getDatabases();
size_t max_queue_size = 0;
size_t max_inserts_in_queue = 0;
size_t max_merges_in_queue = 0;
size_t sum_queue_size = 0;
size_t sum_inserts_in_queue = 0;
size_t sum_merges_in_queue = 0;
size_t max_absolute_delay = 0;
size_t max_relative_delay = 0;
size_t max_part_count_for_partition = 0;
size_t number_of_databases = databases.size();
size_t total_number_of_tables = 0;
size_t total_number_of_bytes = 0;
size_t total_number_of_rows = 0;
size_t total_number_of_parts = 0;
for (const auto & db : databases)
{
/// Check if database can contain MergeTree tables
if (!db.second->canContainMergeTreeTables())
continue;
for (auto iterator = db.second->getTablesIterator(getContext()); iterator->isValid(); iterator->next())
{
++total_number_of_tables;
const auto & table = iterator->table();
if (!table)
continue;
if (MergeTreeData * table_merge_tree = dynamic_cast<MergeTreeData *>(table.get()))
{
const auto & settings = getContext()->getSettingsRef();
calculateMax(max_part_count_for_partition, table_merge_tree->getMaxPartsCountAndSizeForPartition().first);
total_number_of_bytes += table_merge_tree->totalBytes(settings).value();
total_number_of_rows += table_merge_tree->totalRows(settings).value();
total_number_of_parts += table_merge_tree->getPartsCount();
}
if (StorageReplicatedMergeTree * table_replicated_merge_tree = typeid_cast<StorageReplicatedMergeTree *>(table.get()))
{
StorageReplicatedMergeTree::Status status;
table_replicated_merge_tree->getStatus(status, false);
calculateMaxAndSum(max_queue_size, sum_queue_size, status.queue.queue_size);
calculateMaxAndSum(max_inserts_in_queue, sum_inserts_in_queue, status.queue.inserts_in_queue);
calculateMaxAndSum(max_merges_in_queue, sum_merges_in_queue, status.queue.merges_in_queue);
if (!status.is_readonly)
{
try
{
time_t absolute_delay = 0;
time_t relative_delay = 0;
table_replicated_merge_tree->getReplicaDelays(absolute_delay, relative_delay);
calculateMax(max_absolute_delay, absolute_delay);
calculateMax(max_relative_delay, relative_delay);
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__,
"Cannot get replica delay for table: " + backQuoteIfNeed(db.first) + "." + backQuoteIfNeed(iterator->name()));
}
}
}
}
}
new_values["ReplicasMaxQueueSize"] = { max_queue_size, "Maximum queue size (in the number of operations like get, merge) across Replicated tables." };
new_values["ReplicasMaxInsertsInQueue"] = { max_inserts_in_queue, "Maximum number of INSERT operations in the queue (still to be replicated) across Replicated tables." };
new_values["ReplicasMaxMergesInQueue"] = { max_merges_in_queue, "Maximum number of merge operations in the queue (still to be applied) across Replicated tables." };
new_values["ReplicasSumQueueSize"] = { sum_queue_size, "Sum queue size (in the number of operations like get, merge) across Replicated tables." };
new_values["ReplicasSumInsertsInQueue"] = { sum_inserts_in_queue, "Sum of INSERT operations in the queue (still to be replicated) across Replicated tables." };
new_values["ReplicasSumMergesInQueue"] = { sum_merges_in_queue, "Sum of merge operations in the queue (still to be applied) across Replicated tables." };
new_values["ReplicasMaxAbsoluteDelay"] = { max_absolute_delay, "Maximum difference in seconds between the most fresh replicated part and the most fresh data part still to be replicated, across Replicated tables. A very high value indicates a replica with no data." };
new_values["ReplicasMaxRelativeDelay"] = { max_relative_delay, "Maximum difference between the replica delay and the delay of the most up-to-date replica of the same table, across Replicated tables." };
new_values["MaxPartCountForPartition"] = { max_part_count_for_partition, "Maximum number of parts per partition across all partitions of all tables of MergeTree family. Values larger than 300 indicates misconfiguration, overload, or massive data loading." };
new_values["NumberOfDatabases"] = { number_of_databases, "Total number of databases on the server." };
new_values["NumberOfTables"] = { total_number_of_tables, "Total number of tables summed across the databases on the server, excluding the databases that cannot contain MergeTree tables."
" The excluded database engines are those who generate the set of tables on the fly, like `Lazy`, `MySQL`, `PostgreSQL`, `SQlite`."};
new_values["TotalBytesOfMergeTreeTables"] = { total_number_of_bytes, "Total amount of bytes (compressed, including data and indices) stored in all tables of MergeTree family." };
new_values["TotalRowsOfMergeTreeTables"] = { total_number_of_rows, "Total amount of rows (records) stored in all tables of MergeTree family." };
new_values["TotalPartsOfMergeTreeTables"] = { total_number_of_parts, "Total amount of data parts in all tables of MergeTree family."
" Numbers larger than 10 000 will negatively affect the server startup time and it may indicate unreasonable choice of the partition key." };
auto get_metric_name_doc = [](const String & name) -> std::pair<const char *, const char *>
{
static std::map<String, std::pair<const char *, const char *>> metric_map =
@ -1691,7 +1416,9 @@ void AsynchronousMetrics::update(TimePoint update_time)
{"mysql_port", {"MySQLThreads", "Number of threads in the server of the MySQL compatibility protocol."}},
{"postgresql_port", {"PostgreSQLThreads", "Number of threads in the server of the PostgreSQL compatibility protocol."}},
{"grpc_port", {"GRPCThreads", "Number of threads in the server of the GRPC protocol."}},
{"prometheus.port", {"PrometheusThreads", "Number of threads in the server of the Prometheus endpoint. Note: prometheus endpoints can be also used via the usual HTTP/HTTPs ports."}}
{"prometheus.port", {"PrometheusThreads", "Number of threads in the server of the Prometheus endpoint. Note: prometheus endpoints can be also used via the usual HTTP/HTTPs ports."}},
{"keeper_server.tcp_port", {"KeeperTCPThreads", "Number of threads in the server of the Keeper TCP protocol (without TLS)."}},
{"keeper_server.tcp_port_secure", {"KeeperTCPSecureThreads", "Number of threads in the server of the Keeper TCP protocol (with TLS)."}}
};
auto it = metric_map.find(name);
if (it == metric_map.end())
@ -1707,102 +1434,14 @@ void AsynchronousMetrics::update(TimePoint update_time)
new_values[name_doc.first] = { server_metric.current_threads, name_doc.second };
}
}
#if USE_NURAFT
{
auto keeper_dispatcher = getContext()->tryGetKeeperDispatcher();
if (keeper_dispatcher)
{
size_t is_leader = 0;
size_t is_follower = 0;
size_t is_observer = 0;
size_t is_standalone = 0;
size_t znode_count = 0;
size_t watch_count = 0;
size_t ephemerals_count = 0;
size_t approximate_data_size = 0;
size_t key_arena_size = 0;
size_t latest_snapshot_size = 0;
size_t open_file_descriptor_count = 0;
size_t max_file_descriptor_count = 0;
size_t followers = 0;
size_t synced_followers = 0;
size_t zxid = 0;
size_t session_with_watches = 0;
size_t paths_watched = 0;
size_t snapshot_dir_size = 0;
size_t log_dir_size = 0;
if (keeper_dispatcher->isServerActive())
{
auto keeper_info = keeper_dispatcher -> getKeeper4LWInfo();
is_standalone = static_cast<size_t>(keeper_info.is_standalone);
is_leader = static_cast<size_t>(keeper_info.is_leader);
is_observer = static_cast<size_t>(keeper_info.is_observer);
is_follower = static_cast<size_t>(keeper_info.is_follower);
zxid = keeper_info.last_zxid;
const auto & state_machine = keeper_dispatcher->getStateMachine();
znode_count = state_machine.getNodesCount();
watch_count = state_machine.getTotalWatchesCount();
ephemerals_count = state_machine.getTotalEphemeralNodesCount();
approximate_data_size = state_machine.getApproximateDataSize();
key_arena_size = state_machine.getKeyArenaSize();
latest_snapshot_size = state_machine.getLatestSnapshotBufSize();
session_with_watches = state_machine.getSessionsWithWatchesCount();
paths_watched = state_machine.getWatchedPathsCount();
snapshot_dir_size = keeper_dispatcher->getSnapDirSize();
log_dir_size = keeper_dispatcher->getLogDirSize();
#if defined(__linux__) || defined(__APPLE__)
open_file_descriptor_count = getCurrentProcessFDCount();
max_file_descriptor_count = getMaxFileDescriptorCount();
#endif
if (keeper_info.is_leader)
{
followers = keeper_info.follower_count;
synced_followers = keeper_info.synced_follower_count;
}
}
new_values["KeeperIsLeader"] = { is_leader, "1 if ClickHouse Keeper is a leader, 0 otherwise." };
new_values["KeeperIsFollower"] = { is_follower, "1 if ClickHouse Keeper is a follower, 0 otherwise." };
new_values["KeeperIsObserver"] = { is_observer, "1 if ClickHouse Keeper is an observer, 0 otherwise." };
new_values["KeeperIsStandalone"] = { is_standalone, "1 if ClickHouse Keeper is in a standalone mode, 0 otherwise." };
new_values["KeeperZnodeCount"] = { znode_count, "The number of nodes (data entries) in ClickHouse Keeper." };
new_values["KeeperWatchCount"] = { watch_count, "The number of watches in ClickHouse Keeper." };
new_values["KeeperEphemeralsCount"] = { ephemerals_count, "The number of ephemeral nodes in ClickHouse Keeper." };
new_values["KeeperApproximateDataSize"] = { approximate_data_size, "The approximate data size of ClickHouse Keeper, in bytes." };
new_values["KeeperKeyArenaSize"] = { key_arena_size, "The size in bytes of the memory arena for keys in ClickHouse Keeper." };
new_values["KeeperLatestSnapshotSize"] = { latest_snapshot_size, "The uncompressed size in bytes of the latest snapshot created by ClickHouse Keeper." };
new_values["KeeperOpenFileDescriptorCount"] = { open_file_descriptor_count, "The number of open file descriptors in ClickHouse Keeper." };
new_values["KeeperMaxFileDescriptorCount"] = { max_file_descriptor_count, "The maximum number of open file descriptors in ClickHouse Keeper." };
new_values["KeeperFollowers"] = { followers, "The number of followers of ClickHouse Keeper." };
new_values["KeeperSyncedFollowers"] = { synced_followers, "The number of followers of ClickHouse Keeper who are also in-sync." };
new_values["KeeperZxid"] = { zxid, "The current transaction id number (zxid) in ClickHouse Keeper." };
new_values["KeeperSessionWithWatches"] = { session_with_watches, "The number of client sessions of ClickHouse Keeper having watches." };
new_values["KeeperPathsWatched"] = { paths_watched, "The number of different paths watched by the clients of ClickHouse Keeper." };
new_values["KeeperSnapshotDirSize"] = { snapshot_dir_size, "The size of the snapshots directory of ClickHouse Keeper, in bytes." };
new_values["KeeperLogDirSize"] = { log_dir_size, "The size of the logs directory of ClickHouse Keeper, in bytes." };
}
}
#endif
updateHeavyMetricsIfNeeded(current_time, update_time, new_values);
/// Add more metrics as you wish.
updateImpl(new_values, update_time, current_time);
new_values["AsynchronousMetricsCalculationTimeSpent"] = { watch.elapsedSeconds(), "Time in seconds spent for calculation of asynchronous metrics (this is the overhead of asynchronous metrics)." };
/// Log the new metrics.
if (auto asynchronous_metric_log = getContext()->getAsynchronousMetricLog())
{
asynchronous_metric_log->addValues(new_values);
}
logImpl(new_values);
first_run = false;
@ -1811,75 +1450,4 @@ void AsynchronousMetrics::update(TimePoint update_time)
values = new_values;
}
void AsynchronousMetrics::updateDetachedPartsStats()
{
DetachedPartsStats current_values{};
for (const auto & db : DatabaseCatalog::instance().getDatabases())
{
if (!db.second->canContainMergeTreeTables())
continue;
for (auto iterator = db.second->getTablesIterator(getContext()); iterator->isValid(); iterator->next())
{
const auto & table = iterator->table();
if (!table)
continue;
if (MergeTreeData * table_merge_tree = dynamic_cast<MergeTreeData *>(table.get()))
{
for (const auto & detached_part: table_merge_tree->getDetachedParts())
{
if (!detached_part.valid_name)
continue;
if (detached_part.prefix.empty())
++current_values.detached_by_user;
++current_values.count;
}
}
}
}
detached_parts_stats = current_values;
}
void AsynchronousMetrics::updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, AsynchronousMetricValues & new_values)
{
const auto time_after_previous_update = current_time - heavy_metric_previous_update_time;
const bool update_heavy_metric = time_after_previous_update >= heavy_metric_update_period || first_run;
if (update_heavy_metric)
{
heavy_metric_previous_update_time = update_time;
Stopwatch watch;
/// Test shows that listing 100000 entries consuming around 0.15 sec.
updateDetachedPartsStats();
watch.stop();
/// Normally heavy metrics don't delay the rest of the metrics calculation
/// otherwise log the warning message
auto log_level = std::make_pair(DB::LogsLevel::trace, Poco::Message::PRIO_TRACE);
if (watch.elapsedSeconds() > (update_period.count() / 2.))
log_level = std::make_pair(DB::LogsLevel::debug, Poco::Message::PRIO_DEBUG);
else if (watch.elapsedSeconds() > (update_period.count() / 4. * 3))
log_level = std::make_pair(DB::LogsLevel::warning, Poco::Message::PRIO_WARNING);
LOG_IMPL(log, log_level.first, log_level.second,
"Update heavy metrics. "
"Update period {} sec. "
"Update heavy metrics period {} sec. "
"Heavy metrics calculation elapsed: {} sec.",
update_period.count(),
heavy_metric_update_period.count(),
watch.elapsedSeconds());
}
new_values["NumberOfDetachedParts"] = { detached_parts_stats.count, "The total number of parts detached from MergeTree tables. A part can be detached by a user with the `ALTER TABLE DETACH` query or by the server itself it the part is broken, unexpected or unneeded. The server does not care about detached parts and they can be removed." };
new_values["NumberOfDetachedByUserParts"] = { detached_parts_stats.detached_by_user, "The total number of parts detached from MergeTree tables by users with the `ALTER TABLE DETACH` query (as opposed to unexpected, broken or ignored parts). The server does not care about detached parts and they can be removed." };
}
}

View File

@ -1,6 +1,5 @@
#pragma once
#include <Interpreters/Context_fwd.h>
#include <Common/MemoryStatisticsOS.h>
#include <Common/ThreadPool.h>
#include <Common/Stopwatch.h>
@ -55,17 +54,15 @@ struct ProtocolServerMetrics
* All the values are either gauge type (like the total number of tables, the current memory usage).
* Or delta-counters representing some accumulation during the interval of time.
*/
class AsynchronousMetrics : WithContext
class AsynchronousMetrics
{
public:
using ProtocolServerMetricsFunc = std::function<std::vector<ProtocolServerMetrics>()>;
AsynchronousMetrics(
ContextPtr global_context_,
int update_period_seconds,
int heavy_metrics_update_period_seconds,
const ProtocolServerMetricsFunc & protocol_server_metrics_func_);
~AsynchronousMetrics();
virtual ~AsynchronousMetrics();
/// Separate method allows to initialize the `servers` variable beforehand.
void start();
@ -75,12 +72,22 @@ public:
/// Returns copy of all values.
AsynchronousMetricValues getValues() const;
private:
protected:
using Duration = std::chrono::seconds;
using TimePoint = std::chrono::system_clock::time_point;
const Duration update_period;
const Duration heavy_metric_update_period;
/// Some values are incremental and we have to calculate the difference.
/// On first run we will only collect the values to subtract later.
bool first_run = true;
TimePoint previous_update_time;
Poco::Logger * log;
private:
virtual void updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) = 0;
virtual void logImpl(AsynchronousMetricValues &) {}
ProtocolServerMetricsFunc protocol_server_metrics_func;
mutable std::mutex mutex;
@ -88,20 +95,6 @@ private:
bool quit {false};
AsynchronousMetricValues values;
/// Some values are incremental and we have to calculate the difference.
/// On first run we will only collect the values to subtract later.
bool first_run = true;
TimePoint previous_update_time;
TimePoint heavy_metric_previous_update_time;
struct DetachedPartsStats
{
size_t count;
size_t detached_by_user;
};
DetachedPartsStats detached_parts_stats{};
#if defined(OS_LINUX) || defined(OS_FREEBSD)
MemoryStatisticsOS memory_stat;
#endif
@ -212,11 +205,6 @@ private:
void run();
void update(TimePoint update_time);
void updateDetachedPartsStats();
void updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, AsynchronousMetricValues & new_values);
Poco::Logger * log;
};
}

View File

@ -433,6 +433,15 @@ The server successfully detected this situation and will download merged part fr
M(KeeperSnapshotApplysFailed, "Number of failed snapshot applying")\
M(KeeperReadSnapshot, "Number of snapshot read(serialization)")\
M(KeeperSaveSnapshot, "Number of snapshot save")\
M(KeeperCreateRequest, "Number of create requests")\
M(KeeperRemoveRequest, "Number of remove requests")\
M(KeeperSetRequest, "Number of set requests")\
M(KeeperCheckRequest, "Number of check requests")\
M(KeeperMultiRequest, "Number of multi requests")\
M(KeeperMultiReadRequest, "Number of multi read requests")\
M(KeeperGetRequest, "Number of get requests")\
M(KeeperListRequest, "Number of list requests")\
M(KeeperExistsRequest, "Number of exists requests")\
\
M(OverflowBreak, "Number of times, data processing was cancelled by query complexity limitation with setting '*_overflow_mode' = 'break' and the result is incomplete.") \
M(OverflowThrow, "Number of times, data processing was cancelled by query complexity limitation with setting '*_overflow_mode' = 'throw' and exception was thrown.") \

View File

@ -42,15 +42,15 @@ public:
return s;
}
template <typename ValueType, typename ParseFunction>
template <typename ValueType, bool ReturnDefault, typename ParseFunction>
static ValueType getValue(const Node * node, const std::string & path,
const std::optional<ValueType> & default_value, const ParseFunction & parse_function)
const ValueType & default_value, const ParseFunction & parse_function)
{
const auto * value_node = node->getNodeByPath(path);
if (!value_node)
{
if (default_value)
return *default_value;
if constexpr (ReturnDefault)
return default_value;
else
throw Poco::NotFoundException(path);
}
@ -59,34 +59,64 @@ public:
};
std::string getString(const Node * node, const std::string & path, const std::optional<std::string> & default_value)
std::string getString(const Node * node, const std::string & path)
{
return ParseHelper::getValue<std::string>(node, path, default_value, ParseHelper::parseString);
return ParseHelper::getValue<std::string, false>(node, path, {}, ParseHelper::parseString);
}
Int64 getInt64(const Node * node, const std::string & path, const std::optional<Int64> & default_value)
std::string getString(const Node * node, const std::string & path, const std::string & default_value)
{
return ParseHelper::getValue<Int64>(node, path, default_value, ParseHelper::parseInt64);
return ParseHelper::getValue<std::string, true>(node, path, default_value, ParseHelper::parseString);
}
UInt64 getUInt64(const Node * node, const std::string & path, const std::optional<UInt64> & default_value)
Int64 getInt64(const Node * node, const std::string & path)
{
return ParseHelper::getValue<UInt64>(node, path, default_value, ParseHelper::parseUInt64);
return ParseHelper::getValue<Int64, false>(node, path, {}, ParseHelper::parseInt64);
}
int getInt(const Node * node, const std::string & path, const std::optional<int> & default_value)
Int64 getInt64(const Node * node, const std::string & path, Int64 default_value)
{
return ParseHelper::getValue<int>(node, path, default_value, ParseHelper::parseInt);
return ParseHelper::getValue<Int64, true>(node, path, default_value, ParseHelper::parseInt64);
}
unsigned getUInt(const Node * node, const std::string & path, const std::optional<unsigned> & default_value)
UInt64 getUInt64(const Node * node, const std::string & path)
{
return ParseHelper::getValue<unsigned>(node, path, default_value, ParseHelper::parseUInt);
return ParseHelper::getValue<UInt64, false>(node, path, {}, ParseHelper::parseUInt64);
}
bool getBool(const Node * node, const std::string & path, const std::optional<bool> & default_value)
UInt64 getUInt64(const Node * node, const std::string & path, UInt64 default_value)
{
return ParseHelper::getValue<bool>(node, path, default_value, ParseHelper::parseBool);
return ParseHelper::getValue<UInt64, true>(node, path, default_value, ParseHelper::parseUInt64);
}
int getInt(const Node * node, const std::string & path)
{
return ParseHelper::getValue<int, false>(node, path, {}, ParseHelper::parseInt);
}
int getInt(const Node * node, const std::string & path, int default_value)
{
return ParseHelper::getValue<int, true>(node, path, default_value, ParseHelper::parseInt);
}
unsigned getUInt(const Node * node, const std::string & path)
{
return ParseHelper::getValue<unsigned, false>(node, path, {}, ParseHelper::parseUInt);
}
unsigned getUInt(const Node * node, const std::string & path, unsigned default_value)
{
return ParseHelper::getValue<unsigned, true>(node, path, default_value, ParseHelper::parseUInt);
}
bool getBool(const Node * node, const std::string & path)
{
return ParseHelper::getValue<bool, false>(node, path, {}, ParseHelper::parseBool);
}
bool getBool(const Node * node, const std::string & path, bool default_value)
{
return ParseHelper::getValue<bool, true>(node, path, default_value, ParseHelper::parseBool);
}
}

View File

@ -7,17 +7,26 @@
namespace DB:: XMLUtils
{
/// Returns root element of the document.
Poco::XML::Node * getRootNode(Poco::XML::Document * document);
std::string getString(const Poco::XML::Node * node, const std::string & path, const std::optional<std::string> & default_value = std::nullopt);
/// Finds the element in the node's subtree by the specified path and returns its inner text
/// trying to parse it as the requested type.
/// Throws an exception if path is not found.
std::string getString(const Poco::XML::Node * node, const std::string & path);
Int64 getInt64(const Poco::XML::Node * node, const std::string & path);
UInt64 getUInt64(const Poco::XML::Node * node, const std::string & path);
int getInt(const Poco::XML::Node * node, const std::string & path);
unsigned getUInt(const Poco::XML::Node * node, const std::string & path);
bool getBool(const Poco::XML::Node * node, const std::string & path);
Int64 getInt64(const Poco::XML::Node * node, const std::string & path, const std::optional<Int64> & default_value = std::nullopt);
UInt64 getUInt64(const Poco::XML::Node * node, const std::string & path, const std::optional<UInt64> & default_value = std::nullopt);
int getInt(const Poco::XML::Node * node, const std::string & path, const std::optional<int> & default_value = std::nullopt);
unsigned getUInt(const Poco::XML::Node * node, const std::string & path, const std::optional<unsigned> & default_value = std::nullopt);
bool getBool(const Poco::XML::Node * node, const std::string & path, const std::optional<bool> & default_value = std::nullopt);
/// Finds the element in the node's subtree by the specified path and returns its inner text
/// trying to parse it as the requested type.
/// Returns the specified default value if path is not found.
std::string getString(const Poco::XML::Node * node, const std::string & path, const std::string & default_value);
Int64 getInt64(const Poco::XML::Node * node, const std::string & path, Int64 default_value);
UInt64 getUInt64(const Poco::XML::Node * node, const std::string & path, UInt64 default_value);
int getInt(const Poco::XML::Node * node, const std::string & path, int default_value);
unsigned getUInt(const Poco::XML::Node * node, const std::string & path, unsigned default_value);
bool getBool(const Poco::XML::Node * node, const std::string & path, bool default_value);
}

View File

@ -466,7 +466,7 @@ void ZooKeeper::connect(
}
else
{
LOG_TEST(log, "Connected to ZooKeeper at {} with session_id {}{}", socket.peerAddress().toString(), session_id, fail_reasons.str());
LOG_INFO(log, "Connected to ZooKeeper at {} with session_id {}{}", socket.peerAddress().toString(), session_id, fail_reasons.str());
}
}
@ -867,12 +867,12 @@ void ZooKeeper::finalize(bool error_send, bool error_receive, const String & rea
/// If some thread (send/receive) already finalizing session don't try to do it
bool already_started = finalization_started.test_and_set();
LOG_TEST(log, "Finalizing session {}: finalization_started={}, queue_finished={}, reason={}",
session_id, already_started, requests_queue.isFinished(), reason);
if (already_started)
return;
LOG_INFO(log, "Finalizing session {}: finalization_started={}, queue_finished={}, reason={}",
session_id, already_started, requests_queue.isFinished(), reason);
auto expire_session_if_not_expired = [&]
{
/// No new requests will appear in queue after finish()

View File

@ -64,11 +64,11 @@ bool enoughSpaceInDirectory(const std::string & path, size_t data_size)
return data_size <= free_space;
}
std::unique_ptr<TemporaryFile> createTemporaryFile(const std::string & path)
std::unique_ptr<PocoTemporaryFile> createTemporaryFile(const std::string & folder_path)
{
ProfileEvents::increment(ProfileEvents::ExternalProcessingFilesTotal);
fs::create_directories(path);
return std::make_unique<TemporaryFile>(path);
fs::create_directories(folder_path);
return std::make_unique<PocoTemporaryFile>(folder_path);
}
#if !defined(OS_LINUX)

View File

@ -14,10 +14,10 @@ namespace fs = std::filesystem;
namespace DB
{
using TemporaryFile = Poco::TemporaryFile;
using PocoTemporaryFile = Poco::TemporaryFile;
bool enoughSpaceInDirectory(const std::string & path, size_t data_size);
std::unique_ptr<TemporaryFile> createTemporaryFile(const std::string & path);
std::unique_ptr<PocoTemporaryFile> createTemporaryFile(const std::string & folder_path);
// Determine what block device is responsible for specified path

View File

@ -0,0 +1,127 @@
#include <Coordination/KeeperAsynchronousMetrics.h>
#include <Coordination/KeeperDispatcher.h>
#include <Common/getCurrentProcessFDCount.h>
#include <Common/getMaxFileDescriptorCount.h>
namespace DB
{
void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousMetricValues & new_values)
{
#if USE_NURAFT
size_t is_leader = 0;
size_t is_follower = 0;
size_t is_observer = 0;
size_t is_standalone = 0;
size_t znode_count = 0;
size_t watch_count = 0;
size_t ephemerals_count = 0;
size_t approximate_data_size = 0;
size_t key_arena_size = 0;
size_t latest_snapshot_size = 0;
size_t open_file_descriptor_count = 0;
size_t max_file_descriptor_count = 0;
size_t followers = 0;
size_t synced_followers = 0;
size_t zxid = 0;
size_t session_with_watches = 0;
size_t paths_watched = 0;
size_t snapshot_dir_size = 0;
size_t log_dir_size = 0;
if (keeper_dispatcher.isServerActive())
{
auto keeper_info = keeper_dispatcher.getKeeper4LWInfo();
is_standalone = static_cast<size_t>(keeper_info.is_standalone);
is_leader = static_cast<size_t>(keeper_info.is_leader);
is_observer = static_cast<size_t>(keeper_info.is_observer);
is_follower = static_cast<size_t>(keeper_info.is_follower);
zxid = keeper_info.last_zxid;
const auto & state_machine = keeper_dispatcher.getStateMachine();
znode_count = state_machine.getNodesCount();
watch_count = state_machine.getTotalWatchesCount();
ephemerals_count = state_machine.getTotalEphemeralNodesCount();
approximate_data_size = state_machine.getApproximateDataSize();
key_arena_size = state_machine.getKeyArenaSize();
latest_snapshot_size = state_machine.getLatestSnapshotBufSize();
session_with_watches = state_machine.getSessionsWithWatchesCount();
paths_watched = state_machine.getWatchedPathsCount();
snapshot_dir_size = keeper_dispatcher.getSnapDirSize();
log_dir_size = keeper_dispatcher.getLogDirSize();
# if defined(__linux__) || defined(__APPLE__)
open_file_descriptor_count = getCurrentProcessFDCount();
max_file_descriptor_count = getMaxFileDescriptorCount();
# endif
if (keeper_info.is_leader)
{
followers = keeper_info.follower_count;
synced_followers = keeper_info.synced_follower_count;
}
}
new_values["KeeperIsLeader"] = { is_leader, "1 if ClickHouse Keeper is a leader, 0 otherwise." };
new_values["KeeperIsFollower"] = { is_follower, "1 if ClickHouse Keeper is a follower, 0 otherwise." };
new_values["KeeperIsObserver"] = { is_observer, "1 if ClickHouse Keeper is an observer, 0 otherwise." };
new_values["KeeperIsStandalone"] = { is_standalone, "1 if ClickHouse Keeper is in a standalone mode, 0 otherwise." };
new_values["KeeperZnodeCount"] = { znode_count, "The number of nodes (data entries) in ClickHouse Keeper." };
new_values["KeeperWatchCount"] = { watch_count, "The number of watches in ClickHouse Keeper." };
new_values["KeeperEphemeralsCount"] = { ephemerals_count, "The number of ephemeral nodes in ClickHouse Keeper." };
new_values["KeeperApproximateDataSize"] = { approximate_data_size, "The approximate data size of ClickHouse Keeper, in bytes." };
new_values["KeeperKeyArenaSize"] = { key_arena_size, "The size in bytes of the memory arena for keys in ClickHouse Keeper." };
new_values["KeeperLatestSnapshotSize"] = { latest_snapshot_size, "The uncompressed size in bytes of the latest snapshot created by ClickHouse Keeper." };
new_values["KeeperOpenFileDescriptorCount"] = { open_file_descriptor_count, "The number of open file descriptors in ClickHouse Keeper." };
new_values["KeeperMaxFileDescriptorCount"] = { max_file_descriptor_count, "The maximum number of open file descriptors in ClickHouse Keeper." };
new_values["KeeperFollowers"] = { followers, "The number of followers of ClickHouse Keeper." };
new_values["KeeperSyncedFollowers"] = { synced_followers, "The number of followers of ClickHouse Keeper who are also in-sync." };
new_values["KeeperZxid"] = { zxid, "The current transaction id number (zxid) in ClickHouse Keeper." };
new_values["KeeperSessionWithWatches"] = { session_with_watches, "The number of client sessions of ClickHouse Keeper having watches." };
new_values["KeeperPathsWatched"] = { paths_watched, "The number of different paths watched by the clients of ClickHouse Keeper." };
new_values["KeeperSnapshotDirSize"] = { snapshot_dir_size, "The size of the snapshots directory of ClickHouse Keeper, in bytes." };
new_values["KeeperLogDirSize"] = { log_dir_size, "The size of the logs directory of ClickHouse Keeper, in bytes." };
auto keeper_log_info = keeper_dispatcher.getKeeperLogInfo();
new_values["KeeperLastLogIdx"] = { keeper_log_info.last_log_idx, "Index of the last log stored in ClickHouse Keeper." };
new_values["KeeperLastLogTerm"] = { keeper_log_info.last_log_term, "Raft term of the last log stored in ClickHouse Keeper." };
new_values["KeeperLastCommittedLogIdx"] = { keeper_log_info.last_committed_log_idx, "Index of the last committed log in ClickHouse Keeper." };
new_values["KeeperTargetCommitLogIdx"] = { keeper_log_info.target_committed_log_idx, "Index until which logs can be committed in ClickHouse Keeper." };
new_values["KeeperLastSnapshotIdx"] = { keeper_log_info.last_snapshot_idx, "Index of the last log present in the last created snapshot." };
auto & keeper_connection_stats = keeper_dispatcher.getKeeperConnectionStats();
new_values["KeeperMinLatency"] = { keeper_connection_stats.getMinLatency(), "Minimal request latency of ClickHouse Keeper." };
new_values["KeeperMaxLatency"] = { keeper_connection_stats.getMaxLatency(), "Maximum request latency of ClickHouse Keeper." };
new_values["KeeperAvgLatency"] = { keeper_connection_stats.getAvgLatency(), "Average request latency of ClickHouse Keeper." };
new_values["KeeperPacketsReceived"] = { keeper_connection_stats.getPacketsReceived(), "Number of packets received by ClickHouse Keeper." };
new_values["KeeperPacketsSent"] = { keeper_connection_stats.getPacketsSent(), "Number of packets sent by ClickHouse Keeper." };
#endif
}
KeeperAsynchronousMetrics::KeeperAsynchronousMetrics(
TinyContextPtr tiny_context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_)
: AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_), tiny_context(std::move(tiny_context_))
{
}
void KeeperAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values, TimePoint /*update_time*/, TimePoint /*current_time*/)
{
#if USE_NURAFT
{
auto keeper_dispatcher = tiny_context->tryGetKeeperDispatcher();
if (keeper_dispatcher)
updateKeeperInformation(*keeper_dispatcher, new_values);
}
#endif
}
}

View File

@ -0,0 +1,25 @@
#pragma once
#include <Coordination/TinyContext.h>
#include <Common/AsynchronousMetrics.h>
namespace DB
{
class KeeperDispatcher;
void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousMetricValues & new_values);
class KeeperAsynchronousMetrics : public AsynchronousMetrics
{
public:
KeeperAsynchronousMetrics(
TinyContextPtr tiny_context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_);
private:
TinyContextPtr tiny_context;
void updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) override;
};
}

View File

@ -15,6 +15,7 @@
#include <Common/logger_useful.h>
#include <Common/setThreadName.h>
#include <Common/LockMemoryExceptionInThread.h>
#include <Common/ProfileEvents.h>
#include <Coordination/pathUtils.h>
#include <Coordination/KeeperConstants.h>
@ -27,6 +28,19 @@
#include <base/defines.h>
#include <filesystem>
namespace ProfileEvents
{
extern const Event KeeperCreateRequest;
extern const Event KeeperRemoveRequest;
extern const Event KeeperSetRequest;
extern const Event KeeperCheckRequest;
extern const Event KeeperMultiRequest;
extern const Event KeeperMultiReadRequest;
extern const Event KeeperGetRequest;
extern const Event KeeperListRequest;
extern const Event KeeperExistsRequest;
}
namespace DB
{
@ -865,6 +879,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr
std::vector<KeeperStorage::Delta>
preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override
{
ProfileEvents::increment(ProfileEvents::KeeperCreateRequest);
Coordination::ZooKeeperCreateRequest & request = dynamic_cast<Coordination::ZooKeeperCreateRequest &>(*zk_request);
std::vector<KeeperStorage::Delta> new_deltas;
@ -986,6 +1001,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce
std::vector<KeeperStorage::Delta>
preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override
{
ProfileEvents::increment(ProfileEvents::KeeperGetRequest);
Coordination::ZooKeeperGetRequest & request = dynamic_cast<Coordination::ZooKeeperGetRequest &>(*zk_request);
if (request.path == Coordination::keeper_api_version_path)
@ -1040,6 +1056,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce
Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override
{
ProfileEvents::increment(ProfileEvents::KeeperGetRequest);
return processImpl<true>(storage, zxid);
}
};
@ -1055,6 +1072,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr
std::vector<KeeperStorage::Delta>
preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override
{
ProfileEvents::increment(ProfileEvents::KeeperRemoveRequest);
Coordination::ZooKeeperRemoveRequest & request = dynamic_cast<Coordination::ZooKeeperRemoveRequest &>(*zk_request);
std::vector<KeeperStorage::Delta> new_deltas;
@ -1145,6 +1163,7 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr
std::vector<KeeperStorage::Delta>
preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override
{
ProfileEvents::increment(ProfileEvents::KeeperExistsRequest);
Coordination::ZooKeeperExistsRequest & request = dynamic_cast<Coordination::ZooKeeperExistsRequest &>(*zk_request);
if (!storage.uncommitted_state.getNode(request.path))
@ -1194,6 +1213,7 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr
Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override
{
ProfileEvents::increment(ProfileEvents::KeeperExistsRequest);
return processImpl<true>(storage, zxid);
}
};
@ -1209,6 +1229,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce
std::vector<KeeperStorage::Delta>
preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override
{
ProfileEvents::increment(ProfileEvents::KeeperSetRequest);
Coordination::ZooKeeperSetRequest & request = dynamic_cast<Coordination::ZooKeeperSetRequest &>(*zk_request);
std::vector<KeeperStorage::Delta> new_deltas;
@ -1301,6 +1322,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc
std::vector<KeeperStorage::Delta>
preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override
{
ProfileEvents::increment(ProfileEvents::KeeperListRequest);
Coordination::ZooKeeperListRequest & request = dynamic_cast<Coordination::ZooKeeperListRequest &>(*zk_request);
if (!storage.uncommitted_state.getNode(request.path))
@ -1387,6 +1409,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc
Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override
{
ProfileEvents::increment(ProfileEvents::KeeperListRequest);
return processImpl<true>(storage, zxid);
}
};
@ -1402,6 +1425,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro
std::vector<KeeperStorage::Delta>
preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override
{
ProfileEvents::increment(ProfileEvents::KeeperCheckRequest);
Coordination::ZooKeeperCheckRequest & request = dynamic_cast<Coordination::ZooKeeperCheckRequest &>(*zk_request);
if (!storage.uncommitted_state.getNode(request.path))
@ -1463,6 +1487,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro
Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override
{
ProfileEvents::increment(ProfileEvents::KeeperCheckRequest);
return processImpl<true>(storage, zxid);
}
};
@ -1689,6 +1714,7 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro
std::vector<KeeperStorage::Delta>
preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override
{
ProfileEvents::increment(ProfileEvents::KeeperMultiRequest);
std::vector<Coordination::Error> response_errors;
response_errors.reserve(concrete_requests.size());
uint64_t current_digest = digest;
@ -1756,6 +1782,7 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro
Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override
{
ProfileEvents::increment(ProfileEvents::KeeperMultiReadRequest);
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
Coordination::ZooKeeperMultiResponse & response = dynamic_cast<Coordination::ZooKeeperMultiResponse &>(*response_ptr);

View File

@ -1,4 +1,4 @@
#include "TinyContext.h"
#include <Coordination/TinyContext.h>
#include <Common/Exception.h>
#include <Coordination/KeeperDispatcher.h>

View File

@ -10,7 +10,7 @@ namespace DB
class KeeperDispatcher;
class TinyContext: public std::enable_shared_from_this<TinyContext>
class TinyContext : public std::enable_shared_from_this<TinyContext>
{
public:
std::shared_ptr<KeeperDispatcher> getKeeperDispatcher() const;
@ -31,4 +31,6 @@ private:
ConfigurationPtr config TSA_GUARDED_BY(keeper_dispatcher_mutex);
};
using TinyContextPtr = std::shared_ptr<TinyContext>;
}

View File

@ -702,7 +702,18 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
/// We will drop or move tables which exist only in local metadata
Strings tables_to_detach;
std::vector<std::pair<String, String>> replicated_tables_to_rename;
struct RenameEdge
{
String from;
String intermediate;
String to;
};
/// This is needed to generate intermediate name
String salt = toString(thread_local_rng());
std::vector<RenameEdge> replicated_tables_to_rename;
size_t total_tables = 0;
std::vector<UUID> replicated_ids;
for (auto existing_tables_it = getTablesIterator(getContext(), {}); existing_tables_it->isValid();
@ -719,8 +730,15 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
{
if (name != it->second)
{
String intermediate_name;
/// Possibly we failed to rename it on previous iteration
/// And this table was already renamed to an intermediate name
if (startsWith(name, ".rename-") && !startsWith(it->second, ".rename-"))
intermediate_name = name;
else
intermediate_name = fmt::format(".rename-{}-{}", name, sipHash64(fmt::format("{}-{}", name, salt)));
/// Need just update table name
replicated_tables_to_rename.emplace_back(name, it->second);
replicated_tables_to_rename.push_back({name, intermediate_name, it->second});
}
continue;
}
@ -840,13 +858,13 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
tables_to_detach.size(), dropped_dictionaries, dropped_tables.size() - dropped_dictionaries, moved_tables);
/// Now database is cleared from outdated tables, let's rename ReplicatedMergeTree tables to actual names
for (const auto & old_to_new : replicated_tables_to_rename)
/// We have to take into account that tables names could be changed with two general queries
/// 1) RENAME TABLE. There could be multiple pairs of tables (e.g. RENAME b TO c, a TO b, c TO d)
/// But it is equal to multiple subsequent RENAMEs each of which operates only with two tables
/// 2) EXCHANGE TABLE. This query swaps two names atomically and could not be represented with two separate RENAMEs
auto rename_table = [&](String from, String to)
{
const String & from = old_to_new.first;
const String & to = old_to_new.second;
LOG_DEBUG(log, "Will RENAME TABLE {} TO {}", backQuoteIfNeed(from), backQuoteIfNeed(to));
/// TODO Maybe we should do it in two steps: rename all tables to temporary names and then rename them to actual names?
DDLGuardPtr table_guard = DatabaseCatalog::instance().getDDLGuard(db_name, std::min(from, to));
DDLGuardPtr to_table_guard = DatabaseCatalog::instance().getDDLGuard(db_name, std::max(from, to));
@ -858,7 +876,23 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
DatabaseAtomic::renameTable(make_query_context(), from, *this, to, false, false);
tables_metadata_digest = new_digest;
assert(checkDigestValid(getContext()));
};
LOG_DEBUG(log, "Starting first stage of renaming process. Will rename tables to intermediate names");
for (auto & [from, intermediate, _] : replicated_tables_to_rename)
{
/// Due to some unknown failures there could be tables
/// which are already in an intermediate state
/// For them we skip the first stage
if (from == intermediate)
continue;
rename_table(from, intermediate);
}
LOG_DEBUG(log, "Starting second stage of renaming process. Will rename tables from intermediate to desired names");
for (auto & [_, intermediate, to] : replicated_tables_to_rename)
rename_table(intermediate, to);
LOG_DEBUG(log, "Renames completed succesessfully");
for (const auto & id : dropped_tables)
DatabaseCatalog::instance().waitTableFinallyDropped(id);

View File

@ -118,10 +118,7 @@ void CachedOnDiskReadBufferFromFile::initialize(size_t offset, size_t size)
}
else
{
CreateFileSegmentSettings create_settings{
.is_persistent = is_persistent
};
CreateFileSegmentSettings create_settings(is_persistent ? FileSegmentKind::Persistent : FileSegmentKind::Regular);
file_segments_holder.emplace(cache->getOrSet(cache_key, offset, size, create_settings));
}
@ -951,7 +948,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
}
else
{
LOG_TRACE(log, "No space left in cache, will continue without cache download");
LOG_TRACE(log, "No space left in cache to reserve {} bytes, will continue without cache download", size);
file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION);
}

View File

@ -51,16 +51,42 @@ FileSegmentRangeWriter::FileSegmentRangeWriter(
{
}
bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset, bool is_persistent)
bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind)
{
size_t written_size = tryWrite(data, size, offset, segment_kind, true);
return written_size == size;
}
size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind, bool strict)
{
size_t total_written_size = 0;
while (size > 0)
{
size_t written_size = tryWriteImpl(data, size, offset, segment_kind, strict);
chassert(written_size <= size);
if (written_size == 0)
break;
if (data)
data += written_size;
size -= written_size;
offset += written_size;
total_written_size += written_size;
}
return total_written_size;
}
size_t FileSegmentRangeWriter::tryWriteImpl(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind, bool strict)
{
if (finalized)
return false;
return 0;
auto & file_segments = file_segments_holder.file_segments;
if (current_file_segment_it == file_segments.end())
{
current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, is_persistent);
current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, segment_kind);
}
else
{
@ -78,7 +104,7 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset
if (file_segment->range().size() == file_segment->getDownloadedSize())
{
completeFileSegment(*file_segment);
current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, is_persistent);
current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, segment_kind);
}
}
@ -93,20 +119,26 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset
file_segment->completePartAndResetDownloader();
});
bool reserved = file_segment->reserve(size);
if (!reserved)
size_t reserved_size = file_segment->tryReserve(size, strict);
if (reserved_size == 0 || (strict && reserved_size != size))
{
file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION);
appendFilesystemCacheLog(*file_segment);
if (strict)
{
file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION);
appendFilesystemCacheLog(*file_segment);
}
LOG_DEBUG(
&Poco::Logger::get("FileSegmentRangeWriter"),
"Unsuccessful space reservation attempt (size: {}, file segment info: {}",
size, file_segment->getInfoForLog());
return false;
return 0;
}
/// Shrink to reserved size, because we can't write more than reserved
size = reserved_size;
try
{
file_segment->write(data, size, offset);
@ -120,7 +152,17 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset
file_segment->completePartAndResetDownloader();
current_file_segment_write_offset += size;
return true;
return size;
}
bool FileSegmentRangeWriter::reserve(size_t size, size_t offset)
{
return write(nullptr, size, offset, FileSegmentKind::Temporary);
}
size_t FileSegmentRangeWriter::tryReserve(size_t size, size_t offset)
{
return tryWrite(nullptr, size, offset, FileSegmentKind::Temporary);
}
void FileSegmentRangeWriter::finalize()
@ -129,6 +171,7 @@ void FileSegmentRangeWriter::finalize()
return;
auto & file_segments = file_segments_holder.file_segments;
if (file_segments.empty() || current_file_segment_it == file_segments.end())
return;
@ -149,7 +192,7 @@ FileSegmentRangeWriter::~FileSegmentRangeWriter()
}
}
FileSegments::iterator FileSegmentRangeWriter::allocateFileSegment(size_t offset, bool is_persistent)
FileSegments::iterator FileSegmentRangeWriter::allocateFileSegment(size_t offset, FileSegmentKind segment_kind)
{
/**
* Allocate a new file segment starting `offset`.
@ -158,10 +201,7 @@ FileSegments::iterator FileSegmentRangeWriter::allocateFileSegment(size_t offset
std::lock_guard cache_lock(cache->mutex);
CreateFileSegmentSettings create_settings
{
.is_persistent = is_persistent,
};
CreateFileSegmentSettings create_settings(segment_kind);
/// We set max_file_segment_size to be downloaded,
/// if we have less size to write, file segment will be resized in complete() method.
@ -196,12 +236,15 @@ void FileSegmentRangeWriter::appendFilesystemCacheLog(const FileSegment & file_s
}
}
void FileSegmentRangeWriter::completeFileSegment(FileSegment & file_segment)
void FileSegmentRangeWriter::completeFileSegment(FileSegment & file_segment, std::optional<FileSegment::State> state)
{
/// File segment can be detached if space reservation failed.
if (file_segment.isDetached())
return;
if (state.has_value())
file_segment.setDownloadState(*state);
file_segment.completeWithoutState();
appendFilesystemCacheLog(file_segment);
}
@ -270,7 +313,8 @@ void CachedOnDiskWriteBufferFromFile::cacheData(char * data, size_t size)
try
{
if (!cache_writer->write(data, size, current_download_offset, is_persistent_cache_file))
auto segment_kind = is_persistent_cache_file ? FileSegmentKind::Persistent : FileSegmentKind::Regular;
if (!cache_writer->write(data, size, current_download_offset, segment_kind))
{
LOG_INFO(log, "Write-through cache is stopped as cache limit is reached and nothing can be evicted");
return;

View File

@ -4,6 +4,7 @@
#include <IO/WriteSettings.h>
#include <Interpreters/Cache/FileCache.h>
#include <Interpreters/FilesystemCacheLog.h>
#include <Common/filesystemHelpers.h>
namespace Poco
{
@ -28,22 +29,44 @@ public:
FileCache * cache_, const FileSegment::Key & key_,
std::shared_ptr<FilesystemCacheLog> cache_log_, const String & query_id_, const String & source_path_);
/**
* Write a range of file segments. Allocate file segment of `max_file_segment_size` and write to
* it until it is full and then allocate next file segment.
*/
bool write(const char * data, size_t size, size_t offset, bool is_persistent);
/* Write a range of file segments.
* Allocate file segment of `max_file_segment_size` and write to it until it is full and then allocate next file segment.
* If it's impossible to allocate new file segment and reserve space to write all data, then returns false.
*
* Note: the data that was written to file segments before the error occurred is not rolled back.
*/
bool write(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind);
/* Tries to write data to current file segment.
* Size of written data may be less than requested_size, because it may not be enough space.
*
* Returns size of written data.
*/
size_t tryWrite(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind = FileSegmentKind::Regular, bool strict = false);
/// Same as `write/tryWrite`, but doesn't write anything, just reserves some space in cache
bool reserve(size_t size, size_t offset);
size_t tryReserve(size_t size, size_t offset);
void finalize();
size_t currentOffset() const { return current_file_segment_write_offset; }
~FileSegmentRangeWriter();
private:
FileSegments::iterator allocateFileSegment(size_t offset, bool is_persistent);
FileSegments::iterator allocateFileSegment(size_t offset, FileSegmentKind segment_kind);
void appendFilesystemCacheLog(const FileSegment & file_segment);
void completeFileSegment(FileSegment & file_segment);
void completeFileSegment(FileSegment & file_segment, std::optional<FileSegment::State> state = {});
/* Writes data to current file segment as much as possible and returns size of written data, do not allocate new file segments
* In `strict` mode it will write all data or nothing, otherwise it will write as much as possible
* If returned non zero value, then we can try to write again to next file segment.
* If no space is available, returns zero.
*/
size_t tryWriteImpl(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind, bool strict);
FileCache * cache;
FileSegment::Key key;

View File

@ -0,0 +1,73 @@
#include <Disks/IO/FileCachePlaceholder.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_ENOUGH_SPACE;
}
void ISpacePlaceholder::reserveCapacity(size_t requested_capacity)
{
chassert(used_space <= capacity);
size_t remaining_space = capacity - used_space;
LOG_TEST(&Poco::Logger::get("ISpacePlaceholder"), "Reserving {} bytes (used_space: {}, capacity: {})", requested_capacity, used_space, capacity);
if (requested_capacity <= remaining_space)
return;
size_t capacity_to_reserve = requested_capacity - remaining_space;
reserveImpl(capacity_to_reserve);
capacity += capacity_to_reserve;
}
void ISpacePlaceholder::setUsed(size_t size)
{
LOG_TEST(&Poco::Logger::get("ISpacePlaceholder"), "Using {} bytes ({} already used, {} capacity)", size, used_space, capacity);
if (used_space + size > capacity)
{
LOG_WARNING(&Poco::Logger::get("ISpacePlaceholder"), "Used space is greater than capacity. It may lead to not enough space error");
reserveCapacity(size);
}
used_space = used_space + size;
}
FileCachePlaceholder::FileCachePlaceholder(FileCache * cache, const String & name)
: key_name(name)
, file_cache(cache)
{
}
void FileCachePlaceholder::reserveImpl(size_t requested_size)
{
/// We create new cache_writer and will try to reserve requested_size in it
String key = fmt::format("{}_{}", key_name, cache_writers.size());
auto cache_writer = std::make_unique<FileSegmentRangeWriter>(file_cache,
file_cache->hash(key),
/* cache_log_ */ nullptr,
/* query_id_ */ "",
/* source_path_ */ key);
size_t current_offset = cache_writer->currentOffset();
size_t reserved_size = cache_writer->tryReserve(requested_size, current_offset);
if (reserved_size != requested_size)
{
throw Exception(ErrorCodes::NOT_ENOUGH_SPACE,
"Cannot reserve space in file cache "
"({} bytes required, got {} reserved "
"{} / {} bytes used, "
"{} / {} elements used)"
, requested_size, reserved_size
, file_cache->getUsedCacheSize(), file_cache->getTotalMaxSize()
, file_cache->getFileSegmentsNum(), file_cache->getTotalMaxElements());
}
/// Add to cache_writers only if we successfully reserved space, otherwise free reserved_size back
cache_writers.push_back(std::move(cache_writer));
}
}

View File

@ -0,0 +1,61 @@
#pragma once
#include <Interpreters/Cache/FileCache.h>
#include <Disks/IO/CachedOnDiskWriteBufferFromFile.h>
#include <Poco/Logger.h>
#include <Poco/ConsoleChannel.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
/* ISpacePlaceholder is a base class for all classes that need to reserve space in some storage.
* You should resrve space with call reserveCapacity() before writing to it.
* After writing you should call setUsed() to let ISpacePlaceholder know how much space was used.
* It can be different because in some cases you don't know exact size of data you will write (because of compression, for example).
* It's better to reserve more space in advance not to overuse space.
*/
class ISpacePlaceholder
{
public:
/// Reserve space in storage
void reserveCapacity(size_t requested_capacity);
/// Indicate that some space is used
/// It uses reserved space if it is possible, otherwise it reserves more space
void setUsed(size_t size);
virtual ~ISpacePlaceholder() = default;
private:
virtual void reserveImpl(size_t size) = 0;
size_t capacity = 0;
size_t used_space = 0;
};
/* FileCachePlaceholder is a class that reserves space in FileCache.
* Data is written externally, and FileCachePlaceholder is only used to hold space in FileCache.
*/
class FileCachePlaceholder : public ISpacePlaceholder
{
public:
FileCachePlaceholder(FileCache * cache, const String & name);
void reserveImpl(size_t requested_size) override;
private:
std::string key_name;
FileCache * file_cache;
/// On each reserveImpl() call we create new FileSegmentRangeWriter that would be hold space
/// It's required to easily release already reserved space on unsuccessful attempt
std::vector<std::unique_ptr<FileSegmentRangeWriter>> cache_writers;
};
}

View File

@ -113,6 +113,8 @@ public:
WriteSettings getAdjustedSettingsFromMetadataFile(const WriteSettings & settings, const std::string & path) const override;
FileCachePtr getCache() const { return cache; }
private:
FileCache::Key getCacheKey(const std::string & path) const;

View File

@ -519,6 +519,14 @@ void DiskObjectStorage::wrapWithCache(FileCachePtr cache, const FileCacheSetting
object_storage = std::make_shared<CachedObjectStorage>(object_storage, cache, cache_settings, layer_name);
}
FileCachePtr DiskObjectStorage::getCache() const
{
const auto * cached_object_storage = typeid_cast<CachedObjectStorage *>(object_storage.get());
if (!cached_object_storage)
return nullptr;
return cached_object_storage->getCache();
}
NameSet DiskObjectStorage::getCacheLayersNames() const
{
NameSet cache_layers;

View File

@ -186,6 +186,7 @@ public:
/// There can be any number of cache layers:
/// DiskObjectStorage(CachedObjectStorage(...CacheObjectStorage(S3ObjectStorage)...))
void wrapWithCache(FileCachePtr cache, const FileCacheSettings & cache_settings, const String & layer_name);
FileCachePtr getCache() const;
/// Get structure of object storage this disk works with. Examples:
/// DiskObjectStorage(S3ObjectStorage)

View File

@ -480,7 +480,8 @@ void S3ObjectStorage::copyObjectImpl(
auto outcome = client_ptr->CopyObject(request);
if (!outcome.IsSuccess() && outcome.GetError().GetExceptionName() == "EntityTooLarge")
if (!outcome.IsSuccess() && (outcome.GetError().GetExceptionName() == "EntityTooLarge"
|| outcome.GetError().GetExceptionName() == "InvalidRequest"))
{ // Can't come here with MinIO, MinIO allows single part upload for large objects.
copyObjectMultipartImpl(src_bucket, src_key, dst_bucket, dst_key, head, metadata);
return;

View File

@ -0,0 +1,20 @@
#include <Disks/TemporaryFileInPath.h>
#include <Common/filesystemHelpers.h>
namespace DB
{
TemporaryFileInPath::TemporaryFileInPath(const String & folder_path)
: tmp_file(createTemporaryFile(folder_path))
{
chassert(tmp_file);
}
String TemporaryFileInPath::getPath() const
{
return tmp_file->path();
}
TemporaryFileInPath::~TemporaryFileInPath() = default;
}

View File

@ -0,0 +1,21 @@
#pragma once
#include <Disks/TemporaryFileOnDisk.h>
#include <Poco/TemporaryFile.h>
namespace DB
{
/// Wrapper around Poco::TemporaryFile to implement ITemporaryFile.
class TemporaryFileInPath : public ITemporaryFile
{
public:
explicit TemporaryFileInPath(const String & folder_path);
String getPath() const override;
~TemporaryFileInPath() override;
private:
std::unique_ptr<Poco::TemporaryFile> tmp_file;
};
}

View File

@ -2,6 +2,7 @@
#include <Poco/TemporaryFile.h>
#include <Common/CurrentMetrics.h>
#include <Common/logger_useful.h>
#include <Disks/TemporaryFileInPath.h>
#include <filesystem>
@ -15,7 +16,6 @@ namespace CurrentMetrics
extern const Metric TotalTemporaryFiles;
}
namespace DB
{

View File

@ -9,21 +9,30 @@ namespace DB
{
using DiskPtr = std::shared_ptr<IDisk>;
class ITemporaryFile
{
public:
virtual String getPath() const = 0;
virtual ~ITemporaryFile() = default;
};
using TemporaryFileHolder = std::unique_ptr<ITemporaryFile>;
/// This class helps with the handling of temporary files or directories.
/// A unique name for the temporary file or directory is automatically chosen based on a specified prefix.
/// Create a directory in the constructor.
/// The destructor always removes the temporary file or directory with all contained files.
class TemporaryFileOnDisk
class TemporaryFileOnDisk : public ITemporaryFile
{
public:
explicit TemporaryFileOnDisk(const DiskPtr & disk_);
explicit TemporaryFileOnDisk(const DiskPtr & disk_, CurrentMetrics::Value metric_scope);
explicit TemporaryFileOnDisk(const DiskPtr & disk_, const String & prefix);
~TemporaryFileOnDisk();
~TemporaryFileOnDisk() override;
DiskPtr getDisk() const { return disk; }
String getPath() const;
String getPath() const override;
private:
DiskPtr disk;

View File

@ -64,8 +64,10 @@ static void writeData(const ISerialization & serialization, const ColumnPtr & co
}
void NativeWriter::write(const Block & block)
size_t NativeWriter::write(const Block & block)
{
size_t written_before = ostr.count();
/// Additional information about the block.
if (client_revision > 0)
block.info.write(ostr);
@ -161,6 +163,10 @@ void NativeWriter::write(const Block & block)
if (index)
index->blocks.emplace_back(std::move(index_block));
size_t written_after = ostr.count();
size_t written_size = written_after - written_before;
return written_size;
}
}

View File

@ -27,7 +27,9 @@ public:
IndexForNativeFormat * index_ = nullptr, size_t initial_size_of_file_ = 0);
Block getHeader() const { return header; }
void write(const Block & block);
/// Returns the number of bytes written.
size_t write(const Block & block);
void flush();
static String getContentType() { return "application/octet-stream"; }

View File

@ -43,6 +43,16 @@ struct ArrayFirstLastImpl
return array_element;
}
static ColumnPtr createNullableColumn(MutableColumnPtr && column, ColumnUInt8::MutablePtr && null_map)
{
if (auto * nullable_column = typeid_cast<ColumnNullable *>(column.get()))
{
nullable_column->applyNullMap(*null_map);
return std::move(column);
}
return ColumnNullable::create(std::move(column), std::move(null_map));
}
static ColumnPtr execute(const ColumnArray & array, ColumnPtr mapped)
{
const auto * column_filter = typeid_cast<const ColumnUInt8 *>(&*mapped);
@ -94,7 +104,7 @@ struct ArrayFirstLastImpl
}
if constexpr (element_not_exists_strategy == ArrayFirstLastElementNotExistsStrategy::Null)
return ColumnNullable::create(std::move(out), std::move(col_null_map_to));
return createNullableColumn(std::move(out), std::move(col_null_map_to));
return out;
}
@ -106,7 +116,7 @@ struct ArrayFirstLastImpl
if constexpr (element_not_exists_strategy == ArrayFirstLastElementNotExistsStrategy::Null)
{
auto col_null_map_to = ColumnUInt8::create(out->size(), true);
return ColumnNullable::create(std::move(out), std::move(col_null_map_to));
return createNullableColumn(std::move(out), std::move(col_null_map_to));
}
return out;
@ -172,7 +182,7 @@ struct ArrayFirstLastImpl
}
if constexpr (element_not_exists_strategy == ArrayFirstLastElementNotExistsStrategy::Null)
return ColumnNullable::create(std::move(out), std::move(col_null_map_to));
return createNullableColumn(std::move(out), std::move(col_null_map_to));
return out;
}

View File

@ -13,7 +13,7 @@ namespace ErrorCodes
}
WriteBufferFromTemporaryFile::WriteBufferFromTemporaryFile(std::unique_ptr<TemporaryFile> && tmp_file_)
WriteBufferFromTemporaryFile::WriteBufferFromTemporaryFile(std::unique_ptr<PocoTemporaryFile> && tmp_file_)
: WriteBufferFromFile(tmp_file_->path(), DBMS_DEFAULT_BUFFER_SIZE, O_RDWR | O_TRUNC | O_CREAT, 0600), tmp_file(std::move(tmp_file_))
{}
@ -40,11 +40,11 @@ public:
return std::make_shared<ReadBufferFromTemporaryWriteBuffer>(fd, file_name, std::move(origin->tmp_file));
}
ReadBufferFromTemporaryWriteBuffer(int fd_, const std::string & file_name_, std::unique_ptr<TemporaryFile> && tmp_file_)
ReadBufferFromTemporaryWriteBuffer(int fd_, const std::string & file_name_, std::unique_ptr<PocoTemporaryFile> && tmp_file_)
: ReadBufferFromFile(fd_, file_name_), tmp_file(std::move(tmp_file_))
{}
std::unique_ptr<TemporaryFile> tmp_file;
std::unique_ptr<PocoTemporaryFile> tmp_file;
};

View File

@ -20,11 +20,11 @@ public:
~WriteBufferFromTemporaryFile() override;
private:
explicit WriteBufferFromTemporaryFile(std::unique_ptr<TemporaryFile> && tmp_file);
explicit WriteBufferFromTemporaryFile(std::unique_ptr<PocoTemporaryFile> && tmp_file);
std::shared_ptr<ReadBuffer> getReadBufferImpl() override;
std::unique_ptr<TemporaryFile> tmp_file;
std::unique_ptr<PocoTemporaryFile> tmp_file;
friend class ReadBufferFromTemporaryWriteBuffer;
};

View File

@ -5,7 +5,7 @@
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/AsynchronousMetricLog.h>
#include <Interpreters/AsynchronousMetrics.h>
#include <Common/AsynchronousMetrics.h>
namespace DB

View File

@ -1,7 +1,7 @@
#pragma once
#include <Interpreters/SystemLog.h>
#include <Interpreters/AsynchronousMetrics.h>
#include <Common/AsynchronousMetrics.h>
#include <Common/ProfileEvents.h>
#include <Common/CurrentMetrics.h>
#include <Core/NamesAndTypes.h>

View File

@ -2,6 +2,7 @@
#include <Common/randomSeed.h>
#include <Common/SipHash.h>
#include <Common/logger_useful.h>
#include <Interpreters/Cache/FileCacheSettings.h>
#include <Interpreters/Cache/LRUFileCachePriority.h>
#include <IO/ReadHelpers.h>
@ -12,6 +13,7 @@
#include <pcg-random/pcg_random.hpp>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
@ -46,13 +48,27 @@ FileCache::Key FileCache::hash(const String & path)
return Key(sipHash128(path.data(), path.size()));
}
String FileCache::getPathInLocalCache(const Key & key, size_t offset, bool is_persistent) const
String FileCache::getPathInLocalCache(const Key & key, size_t offset, FileSegmentKind segment_kind) const
{
String file_suffix;
switch (segment_kind)
{
case FileSegmentKind::Persistent:
file_suffix = "_persistent";
break;
case FileSegmentKind::Temporary:
file_suffix = "_temporary";
break;
case FileSegmentKind::Regular:
file_suffix = "";
break;
}
auto key_str = key.toString();
return fs::path(cache_base_path)
/ key_str.substr(0, 3)
/ key_str
/ (std::to_string(offset) + (is_persistent ? "_persistent" : ""));
/ (std::to_string(offset) + file_suffix);
}
String FileCache::getPathInLocalCache(const Key & key) const
@ -540,9 +556,6 @@ FileSegmentPtr FileCache::createFileSegmentForDownload(
assertCacheCorrectness(key, cache_lock);
#endif
if (size > max_file_segment_size)
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Requested size exceeds max file segment size");
auto * cell = getCell(key, offset, cache_lock);
if (cell)
throw Exception(
@ -999,9 +1012,17 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard<std::mutex> & cache_lock
fs::directory_iterator key_it{key_prefix_it->path()};
for (; key_it != fs::directory_iterator(); ++key_it)
{
if (!key_it->is_directory())
if (key_it->is_regular_file())
{
LOG_DEBUG(log, "Unexpected file: {}. Expected a directory", key_it->path().string());
if (key_prefix_it->path().filename() == "tmp" && startsWith(key_it->path().filename(), "tmp"))
{
LOG_DEBUG(log, "Found temporary file '{}', will remove it", key_it->path().string());
fs::remove(key_it->path());
}
else
{
LOG_DEBUG(log, "Unexpected file: {}. Expected a directory", key_it->path().string());
}
continue;
}
@ -1009,17 +1030,26 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard<std::mutex> & cache_lock
fs::directory_iterator offset_it{key_it->path()};
for (; offset_it != fs::directory_iterator(); ++offset_it)
{
if (offset_it->is_directory())
{
LOG_DEBUG(log, "Unexpected directory: {}. Expected a file", offset_it->path().string());
continue;
}
auto offset_with_suffix = offset_it->path().filename().string();
auto delim_pos = offset_with_suffix.find('_');
bool parsed;
bool is_persistent = false;
FileSegmentKind segment_kind = FileSegmentKind::Regular;
if (delim_pos == std::string::npos)
parsed = tryParse<UInt64>(offset, offset_with_suffix);
else
{
parsed = tryParse<UInt64>(offset, offset_with_suffix.substr(0, delim_pos));
is_persistent = offset_with_suffix.substr(delim_pos+1) == "persistent";
if (offset_with_suffix.substr(delim_pos+1) == "persistent")
segment_kind = FileSegmentKind::Persistent;
if (offset_with_suffix.substr(delim_pos+1) == "temporary")
segment_kind = FileSegmentKind::Temporary;
}
if (!parsed)
@ -1039,7 +1069,7 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard<std::mutex> & cache_lock
{
auto * cell = addCell(
key, offset, size, FileSegment::State::DOWNLOADED,
CreateFileSegmentSettings{ .is_persistent = is_persistent }, cache_lock);
CreateFileSegmentSettings(segment_kind), cache_lock);
if (cell)
queue_entries.emplace_back(cell->queue_iterator, cell->file_segment);
@ -1151,7 +1181,7 @@ std::vector<String> FileCache::tryGetCachePaths(const Key & key)
for (const auto & [offset, cell] : cells_by_offset)
{
if (cell.file_segment->state() == FileSegment::State::DOWNLOADED)
cache_paths.push_back(getPathInLocalCache(key, offset, cell.file_segment->isPersistent()));
cache_paths.push_back(getPathInLocalCache(key, offset, cell.file_segment->getKind()));
}
return cache_paths;
@ -1173,6 +1203,16 @@ size_t FileCache::getAvailableCacheSizeUnlocked(std::lock_guard<std::mutex> & ca
return max_size - getUsedCacheSizeUnlocked(cache_lock);
}
size_t FileCache::getTotalMaxSize() const
{
return max_size;
}
size_t FileCache::getTotalMaxElements() const
{
return max_element_size;
}
size_t FileCache::getFileSegmentsNum() const
{
std::lock_guard cache_lock(mutex);

View File

@ -80,7 +80,7 @@ public:
static Key hash(const String & path);
String getPathInLocalCache(const Key & key, size_t offset, bool is_persistent) const;
String getPathInLocalCache(const Key & key, size_t offset, FileSegmentKind segment_kind) const;
String getPathInLocalCache(const Key & key) const;
@ -89,8 +89,10 @@ public:
size_t capacity() const { return max_size; }
size_t getUsedCacheSize() const;
size_t getTotalMaxSize() const;
size_t getFileSegmentsNum() const;
size_t getTotalMaxElements() const;
static bool isReadOnly();
@ -221,6 +223,8 @@ private:
FileSegmentCell * getCell(const Key & key, size_t offset, std::lock_guard<std::mutex> & cache_lock);
/// Returns non-owened pointer to the cell stored in the `files` map.
/// Doesn't reserve any space.
FileSegmentCell * addCell(
const Key & key,
size_t offset,

View File

@ -31,14 +31,21 @@ const FileCacheSettings & FileCacheFactory::getSettings(const std::string & cach
}
FileCachePtr FileCacheFactory::get(const std::string & cache_base_path)
FileCachePtr FileCacheFactory::tryGet(const std::string & cache_base_path)
{
std::lock_guard lock(mutex);
auto it = caches_by_path.find(cache_base_path);
if (it == caches_by_path.end())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No cache found by path: {}", cache_base_path);
return nullptr;
return it->second->cache;
}
FileCachePtr FileCacheFactory::get(const std::string & cache_base_path)
{
auto file_cache_ptr = tryGet(cache_base_path);
if (!file_cache_ptr)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No cache found by path: {}", cache_base_path);
return file_cache_ptr;
}
FileCachePtr FileCacheFactory::getOrCreate(

View File

@ -33,6 +33,7 @@ public:
FileCachePtr getOrCreate(const std::string & cache_base_path, const FileCacheSettings & file_cache_settings, const std::string & name);
FileCachePtr tryGet(const std::string & cache_base_path);
FileCachePtr get(const std::string & cache_base_path);
CacheByBasePath getAll();

View File

@ -23,6 +23,19 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
String toString(FileSegmentKind type)
{
switch (type)
{
case FileSegmentKind::Regular:
return "Regular";
case FileSegmentKind::Persistent:
return "Persistent";
case FileSegmentKind::Temporary:
return "Temporary";
}
}
FileSegment::FileSegment(
size_t offset_,
size_t size_,
@ -39,7 +52,7 @@ FileSegment::FileSegment(
#else
, log(&Poco::Logger::get("FileSegment"))
#endif
, is_persistent(settings.is_persistent)
, segment_kind(settings.type)
{
/// On creation, file segment state can be EMPTY, DOWNLOADED, DOWNLOADING.
switch (download_state)
@ -73,7 +86,8 @@ FileSegment::FileSegment(
String FileSegment::getPathInLocalCache() const
{
return cache->getPathInLocalCache(key(), offset(), isPersistent());
chassert(cache);
return cache->getPathInLocalCache(key(), offset(), segment_kind);
}
FileSegment::State FileSegment::state() const
@ -309,7 +323,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset)
if (current_downloaded_size == range().size())
throw Exception(ErrorCodes::LOGICAL_ERROR, "File segment is already fully downloaded");
if (!cache_writer)
if (!cache_writer && from != nullptr)
{
if (current_downloaded_size > 0)
throw Exception(
@ -324,11 +338,14 @@ void FileSegment::write(const char * from, size_t size, size_t offset)
try
{
cache_writer->write(from, size);
/// if `from` is nullptr, then we just allocate and hold space by current segment and it was (or would) be written outside
if (cache_writer && from != nullptr)
cache_writer->write(from, size);
std::unique_lock download_lock(download_mutex);
cache_writer->next();
if (cache_writer && from != nullptr)
cache_writer->next();
downloaded_size += size;
}
@ -379,6 +396,13 @@ FileSegment::State FileSegment::wait()
}
bool FileSegment::reserve(size_t size_to_reserve)
{
size_t reserved = tryReserve(size_to_reserve, true);
assert(reserved == 0 || reserved == size_to_reserve);
return reserved == size_to_reserve;
}
size_t FileSegment::tryReserve(size_t size_to_reserve, bool strict)
{
if (!size_to_reserve)
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Zero space reservation is not allowed");
@ -394,10 +418,16 @@ bool FileSegment::reserve(size_t size_to_reserve)
expected_downloaded_size = getDownloadedSizeUnlocked(segment_lock);
if (expected_downloaded_size + size_to_reserve > range().size())
throw Exception(
ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR,
"Attempt to reserve space too much space ({}) for file segment with range: {} (downloaded size: {})",
size_to_reserve, range().toString(), downloaded_size);
{
if (strict)
{
throw Exception(
ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR,
"Attempt to reserve space too much space ({}) for file segment with range: {} (downloaded size: {})",
size_to_reserve, range().toString(), downloaded_size);
}
size_to_reserve = range().size() - expected_downloaded_size;
}
chassert(reserved_size >= expected_downloaded_size);
}
@ -415,17 +445,16 @@ bool FileSegment::reserve(size_t size_to_reserve)
{
std::lock_guard cache_lock(cache->mutex);
size_to_reserve = size_to_reserve - already_reserved_size;
reserved = cache->tryReserve(key(), offset(), size_to_reserve, cache_lock);
size_t need_to_reserve = size_to_reserve - already_reserved_size;
reserved = cache->tryReserve(key(), offset(), need_to_reserve, cache_lock);
if (reserved)
{
std::lock_guard segment_lock(mutex);
reserved_size += size_to_reserve;
}
if (!reserved)
return 0;
std::lock_guard segment_lock(mutex);
reserved_size += need_to_reserve;
}
return reserved;
return size_to_reserve;
}
void FileSegment::setDownloadedUnlocked([[maybe_unused]] std::unique_lock<std::mutex> & segment_lock)
@ -545,6 +574,15 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard<std::mutex> & cach
resetDownloaderUnlocked(segment_lock);
}
if (segment_kind == FileSegmentKind::Temporary && is_last_holder)
{
LOG_TEST(log, "Removing temporary file segment: {}", getInfoForLogUnlocked(segment_lock));
detach(cache_lock, segment_lock);
setDownloadState(State::SKIP_CACHE);
cache->remove(key(), offset(), cache_lock, segment_lock);
return;
}
switch (download_state)
{
case State::SKIP_CACHE:
@ -626,7 +664,7 @@ String FileSegment::getInfoForLogUnlocked(std::unique_lock<std::mutex> & segment
info << "first non-downloaded offset: " << getFirstNonDownloadedOffsetUnlocked(segment_lock) << ", ";
info << "caller id: " << getCallerId() << ", ";
info << "detached: " << is_detached << ", ";
info << "persistent: " << is_persistent;
info << "kind: " << toString(segment_kind);
return info.str();
}
@ -721,7 +759,7 @@ FileSegmentPtr FileSegment::getSnapshot(const FileSegmentPtr & file_segment, std
snapshot->ref_count = file_segment.use_count();
snapshot->downloaded_size = file_segment->getDownloadedSizeUnlocked(segment_lock);
snapshot->download_state = file_segment->download_state;
snapshot->is_persistent = file_segment->isPersistent();
snapshot->segment_kind = file_segment->getKind();
return snapshot;
}
@ -783,6 +821,8 @@ FileSegmentsHolder::~FileSegmentsHolder()
if (!cache)
cache = file_segment->cache;
assert(cache == file_segment->cache); /// all segments should belong to the same cache
try
{
bool is_detached = false;

View File

@ -30,9 +30,38 @@ using FileSegmentPtr = std::shared_ptr<FileSegment>;
using FileSegments = std::list<FileSegmentPtr>;
/*
* FileSegmentKind is used to specify the eviction policy for file segments.
*/
enum class FileSegmentKind
{
/* `Regular` file segment is still in cache after usage, and can be evicted
* (unless there're some holders).
*/
Regular,
/* `Persistent` file segment can't be evicted from cache,
* it should be removed manually.
*/
Persistent,
/* `Temporary` file segment is removed right after relesing.
* Also corresponding files are removed during cache loading (if any).
*/
Temporary,
};
String toString(FileSegmentKind type);
struct CreateFileSegmentSettings
{
bool is_persistent = false;
FileSegmentKind type = FileSegmentKind::Regular;
CreateFileSegmentSettings() = default;
explicit CreateFileSegmentSettings(FileSegmentKind type_)
: type(type_)
{}
};
class FileSegment : private boost::noncopyable, public std::enable_shared_from_this<FileSegment>
@ -127,7 +156,8 @@ public:
size_t offset() const { return range().left; }
bool isPersistent() const { return is_persistent; }
FileSegmentKind getKind() const { return segment_kind; }
bool isPersistent() const { return segment_kind == FileSegmentKind::Persistent; }
using UniqueId = std::pair<FileCacheKey, size_t>;
UniqueId getUniqueId() const { return std::pair(key(), offset()); }
@ -183,19 +213,19 @@ public:
void assertCorrectness() const;
/**
* ========== Methods for _only_ file segment's `writer` ======================
*/
void synchronousWrite(const char * from, size_t size, size_t offset);
/**
* ========== Methods for _only_ file segment's `downloader` ==================
*/
/// Try to reserve exactly `size` bytes.
/// Returns true if reservation was successful, false otherwise.
bool reserve(size_t size_to_reserve);
/// Try to reserve at max `size` bytes.
/// Returns actual size reserved.
/// In strict mode throws an error on attempt to reserve space too much space
size_t tryReserve(size_t size_to_reserve, bool strict = false);
/// Write data into reserved space.
void write(const char * from, size_t size, size_t offset);
@ -247,9 +277,9 @@ private:
void assertIsDownloaderUnlocked(const std::string & operation, std::unique_lock<std::mutex> & segment_lock) const;
void assertCorrectnessUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
/// complete() without any completion state is called from destructor of
/// FileSegmentsHolder. complete() might check if the caller of the method
/// is the last alive holder of the segment. Therefore, complete() and destruction
/// completeWithoutStateUnlocked() is called from destructor of FileSegmentsHolder.
/// Function might check if the caller of the method
/// is the last alive holder of the segment. Therefore, completion and destruction
/// of the file segment pointer must be done under the same cache mutex.
void completeWithoutStateUnlocked(std::lock_guard<std::mutex> & cache_lock);
void completeBasedOnCurrentState(std::lock_guard<std::mutex> & cache_lock, std::unique_lock<std::mutex> & segment_lock);
@ -295,12 +325,12 @@ private:
/// In general case, all file segments are owned by cache.
bool is_detached = false;
bool is_downloaded{false};
bool is_downloaded = false;
std::atomic<size_t> hits_count = 0; /// cache hits.
std::atomic<size_t> ref_count = 0; /// Used for getting snapshot state
bool is_persistent;
FileSegmentKind segment_kind;
CurrentMetrics::Increment metric_increment{CurrentMetrics::CacheFileSegments};
};
@ -313,6 +343,8 @@ struct FileSegmentsHolder : private boost::noncopyable
FileSegmentsHolder(FileSegmentsHolder && other) noexcept : file_segments(std::move(other.file_segments)) {}
void reset() { file_segments.clear(); }
~FileSegmentsHolder();
String toString();

View File

@ -32,6 +32,7 @@
#include <Storages/StorageS3Settings.h>
#include <Disks/DiskLocal.h>
#include <Disks/DiskDecorator.h>
#include <Disks/ObjectStorages/DiskObjectStorage.h>
#include <Disks/ObjectStorages/IObjectStorage.h>
#include <Disks/IO/ThreadPoolRemoteFSReader.h>
#include <Disks/IO/ThreadPoolReader.h>
@ -102,6 +103,7 @@
#include <Interpreters/Lemmatizers.h>
#include <Interpreters/ClusterDiscovery.h>
#include <Interpreters/TransactionLog.h>
#include <Interpreters/Cache/FileCacheFactory.h>
#include <filesystem>
#include <re2/re2.h>
@ -746,28 +748,65 @@ void Context::setPath(const String & path)
shared->user_scripts_path = shared->path + "user_scripts/";
}
VolumePtr Context::setTemporaryStorage(const String & path, const String & policy_name, size_t max_size)
static void setupTmpPath(Poco::Logger * log, const std::string & path)
try
{
LOG_DEBUG(log, "Setting up {} to store temporary data in it", path);
fs::create_directories(path);
/// Clearing old temporary files.
fs::directory_iterator dir_end;
for (fs::directory_iterator it(path); it != dir_end; ++it)
{
if (it->is_regular_file() && startsWith(it->path().filename(), "tmp"))
{
LOG_DEBUG(log, "Removing old temporary file {}", it->path().string());
fs::remove(it->path());
}
else
LOG_DEBUG(log, "Found unknown file in temporary path {}", it->path().string());
}
}
catch (...)
{
DB::tryLogCurrentException(log, fmt::format(
"Caught exception while setup temporary path: {}. "
"It is ok to skip this exception as cleaning old temporary files is not necessary", path));
}
static VolumePtr createLocalSingleDiskVolume(const std::string & path)
{
auto disk = std::make_shared<DiskLocal>("_tmp_default", path, 0);
VolumePtr volume = std::make_shared<SingleDiskVolume>("_tmp_default", disk, 0);
return volume;
}
void Context::setTemporaryStoragePath(const String & path, size_t max_size)
{
shared->tmp_path = path;
if (!shared->tmp_path.ends_with('/'))
shared->tmp_path += '/';
VolumePtr volume = createLocalSingleDiskVolume(shared->tmp_path);
for (const auto & disk : volume->getDisks())
{
setupTmpPath(shared->log, disk->getPath());
}
shared->temp_data_on_disk = std::make_shared<TemporaryDataOnDiskScope>(volume, nullptr, max_size);
}
void Context::setTemporaryStoragePolicy(const String & policy_name, size_t max_size)
{
std::lock_guard lock(shared->storage_policies_mutex);
VolumePtr volume;
if (policy_name.empty())
{
shared->tmp_path = path;
if (!shared->tmp_path.ends_with('/'))
shared->tmp_path += '/';
auto disk = std::make_shared<DiskLocal>("_tmp_default", shared->tmp_path, 0);
volume = std::make_shared<SingleDiskVolume>("_tmp_default", disk, 0);
}
else
{
StoragePolicyPtr tmp_policy = getStoragePolicySelector(lock)->get(policy_name);
if (tmp_policy->getVolumes().size() != 1)
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG,
"Policy '{}' is used temporary files, such policy should have exactly one volume", policy_name);
volume = tmp_policy->getVolume(0);
}
StoragePolicyPtr tmp_policy = getStoragePolicySelector(lock)->get(policy_name);
if (tmp_policy->getVolumes().size() != 1)
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG,
"Policy '{}' is used temporary files, such policy should have exactly one volume", policy_name);
VolumePtr volume = tmp_policy->getVolume(0);
if (volume->getDisks().empty())
throw Exception("No disks volume for temporary files", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
@ -789,10 +828,33 @@ VolumePtr Context::setTemporaryStorage(const String & path, const String & polic
"Disk '{}' ({}) is not local and can't be used for temporary files",
disk_ptr->getName(), typeid(*disk_raw_ptr).name());
}
setupTmpPath(shared->log, disk->getPath());
}
shared->temp_data_on_disk = std::make_shared<TemporaryDataOnDiskScope>(volume, max_size);
return volume;
shared->temp_data_on_disk = std::make_shared<TemporaryDataOnDiskScope>(volume, nullptr, max_size);
}
void Context::setTemporaryStorageInCache(const String & cache_disk_name, size_t max_size)
{
auto disk_ptr = getDisk(cache_disk_name);
if (!disk_ptr)
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Disk '{}' is not found", cache_disk_name);
const auto * disk_object_storage_ptr = dynamic_cast<const DiskObjectStorage *>(disk_ptr.get());
if (!disk_object_storage_ptr)
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Disk '{}' does not use cache", cache_disk_name);
auto file_cache = disk_object_storage_ptr->getCache();
if (!file_cache)
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Cache '{}' is not found", file_cache->getBasePath());
LOG_DEBUG(shared->log, "Using file cache ({}) for temporary files", file_cache->getBasePath());
shared->tmp_path = file_cache->getBasePath();
VolumePtr volume = createLocalSingleDiskVolume(shared->tmp_path);
shared->temp_data_on_disk = std::make_shared<TemporaryDataOnDiskScope>(volume, file_cache.get(), max_size);
}
void Context::setFlagsPath(const String & path)

View File

@ -19,6 +19,8 @@
#include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h>
#include <Storages/ColumnsDescription.h>
#include <Server/HTTP/HTTPContext.h>
#include "config.h"
@ -461,7 +463,9 @@ public:
void addWarningMessage(const String & msg) const;
VolumePtr setTemporaryStorage(const String & path, const String & policy_name, size_t max_size);
void setTemporaryStorageInCache(const String & cache_disk_name, size_t max_size);
void setTemporaryStoragePolicy(const String & policy_name, size_t max_size);
void setTemporaryStoragePath(const String & path, size_t max_size);
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
@ -1077,4 +1081,53 @@ private:
DiskSelectorPtr getDiskSelector(std::lock_guard<std::mutex> & /* lock */) const;
};
struct HTTPContext : public IHTTPContext
{
explicit HTTPContext(ContextPtr context_)
: context(Context::createCopy(context_))
{}
uint64_t getMaxHstsAge() const override
{
return context->getSettingsRef().hsts_max_age;
}
uint64_t getMaxUriSize() const override
{
return context->getSettingsRef().http_max_uri_size;
}
uint64_t getMaxFields() const override
{
return context->getSettingsRef().http_max_fields;
}
uint64_t getMaxFieldNameSize() const override
{
return context->getSettingsRef().http_max_field_name_size;
}
uint64_t getMaxFieldValueSize() const override
{
return context->getSettingsRef().http_max_field_value_size;
}
uint64_t getMaxChunkSize() const override
{
return context->getSettingsRef().http_max_chunk_size;
}
Poco::Timespan getReceiveTimeout() const override
{
return context->getSettingsRef().http_receive_timeout;
}
Poco::Timespan getSendTimeout() const override
{
return context->getSettingsRef().http_send_timeout;
}
ContextPtr context;
};
}

View File

@ -0,0 +1,395 @@
#include <Interpreters/ServerAsynchronousMetrics.h>
#include <Interpreters/Aggregator.h>
#include <Interpreters/AsynchronousMetricLog.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/Cache/FileCache.h>
#include <Interpreters/Cache/FileCacheFactory.h>
#include <Interpreters/Context.h>
#include <Interpreters/JIT/CompiledExpressionCache.h>
#include <Databases/IDatabase.h>
#include <IO/UncompressedCache.h>
#include <IO/MMappedFileCache.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MergeTreeMetadataCache.h>
#include <Storages/StorageMergeTree.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/MarkCache.h>
#include <Coordination/KeeperAsynchronousMetrics.h>
namespace DB
{
namespace
{
template <typename Max, typename T>
void calculateMax(Max & max, T x)
{
if (Max(x) > max)
max = x;
}
template <typename Max, typename Sum, typename T>
void calculateMaxAndSum(Max & max, Sum & sum, T x)
{
sum += x;
if (Max(x) > max)
max = x;
}
}
ServerAsynchronousMetrics::ServerAsynchronousMetrics(
ContextPtr global_context_,
int update_period_seconds,
int heavy_metrics_update_period_seconds,
const ProtocolServerMetricsFunc & protocol_server_metrics_func_)
: AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_)
, WithContext(global_context_)
, heavy_metric_update_period(heavy_metrics_update_period_seconds)
{}
void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time)
{
if (auto mark_cache = getContext()->getMarkCache())
{
new_values["MarkCacheBytes"] = { mark_cache->weight(), "Total size of mark cache in bytes" };
new_values["MarkCacheFiles"] = { mark_cache->count(), "Total number of mark files cached in the mark cache" };
}
if (auto uncompressed_cache = getContext()->getUncompressedCache())
{
new_values["UncompressedCacheBytes"] = { uncompressed_cache->weight(),
"Total size of uncompressed cache in bytes. Uncompressed cache does not usually improve the performance and should be mostly avoided." };
new_values["UncompressedCacheCells"] = { uncompressed_cache->count(),
"Total number of entries in the uncompressed cache. Each entry represents a decompressed block of data. Uncompressed cache does not usually improve performance and should be mostly avoided." };
}
if (auto index_mark_cache = getContext()->getIndexMarkCache())
{
new_values["IndexMarkCacheBytes"] = { index_mark_cache->weight(), "Total size of mark cache for secondary indices in bytes." };
new_values["IndexMarkCacheFiles"] = { index_mark_cache->count(), "Total number of mark files cached in the mark cache for secondary indices." };
}
if (auto index_uncompressed_cache = getContext()->getIndexUncompressedCache())
{
new_values["IndexUncompressedCacheBytes"] = { index_uncompressed_cache->weight(),
"Total size of uncompressed cache in bytes for secondary indices. Uncompressed cache does not usually improve the performance and should be mostly avoided." };
new_values["IndexUncompressedCacheCells"] = { index_uncompressed_cache->count(),
"Total number of entries in the uncompressed cache for secondary indices. Each entry represents a decompressed block of data. Uncompressed cache does not usually improve performance and should be mostly avoided." };
}
if (auto mmap_cache = getContext()->getMMappedFileCache())
{
new_values["MMapCacheCells"] = { mmap_cache->count(),
"The number of files opened with `mmap` (mapped in memory)."
" This is used for queries with the setting `local_filesystem_read_method` set to `mmap`."
" The files opened with `mmap` are kept in the cache to avoid costly TLB flushes."};
}
{
auto caches = FileCacheFactory::instance().getAll();
size_t total_bytes = 0;
size_t total_files = 0;
for (const auto & [_, cache_data] : caches)
{
total_bytes += cache_data->cache->getUsedCacheSize();
total_files += cache_data->cache->getFileSegmentsNum();
}
new_values["FilesystemCacheBytes"] = { total_bytes,
"Total bytes in the `cache` virtual filesystem. This cache is hold on disk." };
new_values["FilesystemCacheFiles"] = { total_files,
"Total number of cached file segments in the `cache` virtual filesystem. This cache is hold on disk." };
}
#if USE_ROCKSDB
if (auto metadata_cache = getContext()->tryGetMergeTreeMetadataCache())
{
new_values["MergeTreeMetadataCacheSize"] = { metadata_cache->getEstimateNumKeys(),
"The size of the metadata cache for tables. This cache is experimental and not used in production." };
}
#endif
#if USE_EMBEDDED_COMPILER
if (auto * compiled_expression_cache = CompiledExpressionCacheFactory::instance().tryGetCache())
{
new_values["CompiledExpressionCacheBytes"] = { compiled_expression_cache->weight(),
"Total bytes used for the cache of JIT-compiled code." };
new_values["CompiledExpressionCacheCount"] = { compiled_expression_cache->count(),
"Total entries in the cache of JIT-compiled code." };
}
#endif
new_values["Uptime"] = { getContext()->getUptimeSeconds(),
"The server uptime in seconds. It includes the time spent for server initialization before accepting connections." };
if (const auto stats = getHashTablesCacheStatistics())
{
new_values["HashTableStatsCacheEntries"] = { stats->entries,
"The number of entries in the cache of hash table sizes."
" The cache for hash table sizes is used for predictive optimization of GROUP BY." };
new_values["HashTableStatsCacheHits"] = { stats->hits,
"The number of times the prediction of a hash table size was correct." };
new_values["HashTableStatsCacheMisses"] = { stats->misses,
"The number of times the prediction of a hash table size was incorrect." };
}
/// Free space in filesystems at data path and logs path.
{
auto stat = getStatVFS(getContext()->getPath());
new_values["FilesystemMainPathTotalBytes"] = { stat.f_blocks * stat.f_frsize,
"The size of the volume where the main ClickHouse path is mounted, in bytes." };
new_values["FilesystemMainPathAvailableBytes"] = { stat.f_bavail * stat.f_frsize,
"Available bytes on the volume where the main ClickHouse path is mounted." };
new_values["FilesystemMainPathUsedBytes"] = { (stat.f_blocks - stat.f_bavail) * stat.f_frsize,
"Used bytes on the volume where the main ClickHouse path is mounted." };
new_values["FilesystemMainPathTotalINodes"] = { stat.f_files,
"The total number of inodes on the volume where the main ClickHouse path is mounted. If it is less than 25 million, it indicates a misconfiguration." };
new_values["FilesystemMainPathAvailableINodes"] = { stat.f_favail,
"The number of available inodes on the volume where the main ClickHouse path is mounted. If it is close to zero, it indicates a misconfiguration, and you will get 'no space left on device' even when the disk is not full." };
new_values["FilesystemMainPathUsedINodes"] = { stat.f_files - stat.f_favail,
"The number of used inodes on the volume where the main ClickHouse path is mounted. This value mostly corresponds to the number of files." };
}
{
/// Current working directory of the server is the directory with logs.
auto stat = getStatVFS(".");
new_values["FilesystemLogsPathTotalBytes"] = { stat.f_blocks * stat.f_frsize,
"The size of the volume where ClickHouse logs path is mounted, in bytes. It's recommended to have at least 10 GB for logs." };
new_values["FilesystemLogsPathAvailableBytes"] = { stat.f_bavail * stat.f_frsize,
"Available bytes on the volume where ClickHouse logs path is mounted. If this value approaches zero, you should tune the log rotation in the configuration file." };
new_values["FilesystemLogsPathUsedBytes"] = { (stat.f_blocks - stat.f_bavail) * stat.f_frsize,
"Used bytes on the volume where ClickHouse logs path is mounted." };
new_values["FilesystemLogsPathTotalINodes"] = { stat.f_files,
"The total number of inodes on the volume where ClickHouse logs path is mounted." };
new_values["FilesystemLogsPathAvailableINodes"] = { stat.f_favail,
"The number of available inodes on the volume where ClickHouse logs path is mounted." };
new_values["FilesystemLogsPathUsedINodes"] = { stat.f_files - stat.f_favail,
"The number of used inodes on the volume where ClickHouse logs path is mounted." };
}
/// Free and total space on every configured disk.
{
DisksMap disks_map = getContext()->getDisksMap();
for (const auto & [name, disk] : disks_map)
{
auto total = disk->getTotalSpace();
/// Some disks don't support information about the space.
if (!total)
continue;
auto available = disk->getAvailableSpace();
auto unreserved = disk->getUnreservedSpace();
new_values[fmt::format("DiskTotal_{}", name)] = { total,
"The total size in bytes of the disk (virtual filesystem). Remote filesystems can show a large value like 16 EiB." };
new_values[fmt::format("DiskUsed_{}", name)] = { total - available,
"Used bytes on the disk (virtual filesystem). Remote filesystems not always provide this information." };
new_values[fmt::format("DiskAvailable_{}", name)] = { available,
"Available bytes on the disk (virtual filesystem). Remote filesystems can show a large value like 16 EiB." };
new_values[fmt::format("DiskUnreserved_{}", name)] = { unreserved,
"Available bytes on the disk (virtual filesystem) without the reservations for merges, fetches, and moves. Remote filesystems can show a large value like 16 EiB." };
}
}
{
auto databases = DatabaseCatalog::instance().getDatabases();
size_t max_queue_size = 0;
size_t max_inserts_in_queue = 0;
size_t max_merges_in_queue = 0;
size_t sum_queue_size = 0;
size_t sum_inserts_in_queue = 0;
size_t sum_merges_in_queue = 0;
size_t max_absolute_delay = 0;
size_t max_relative_delay = 0;
size_t max_part_count_for_partition = 0;
size_t number_of_databases = databases.size();
size_t total_number_of_tables = 0;
size_t total_number_of_bytes = 0;
size_t total_number_of_rows = 0;
size_t total_number_of_parts = 0;
for (const auto & db : databases)
{
/// Check if database can contain MergeTree tables
if (!db.second->canContainMergeTreeTables())
continue;
for (auto iterator = db.second->getTablesIterator(getContext()); iterator->isValid(); iterator->next())
{
++total_number_of_tables;
const auto & table = iterator->table();
if (!table)
continue;
if (MergeTreeData * table_merge_tree = dynamic_cast<MergeTreeData *>(table.get()))
{
const auto & settings = getContext()->getSettingsRef();
calculateMax(max_part_count_for_partition, table_merge_tree->getMaxPartsCountAndSizeForPartition().first);
total_number_of_bytes += table_merge_tree->totalBytes(settings).value();
total_number_of_rows += table_merge_tree->totalRows(settings).value();
total_number_of_parts += table_merge_tree->getPartsCount();
}
if (StorageReplicatedMergeTree * table_replicated_merge_tree = typeid_cast<StorageReplicatedMergeTree *>(table.get()))
{
StorageReplicatedMergeTree::Status status;
table_replicated_merge_tree->getStatus(status, false);
calculateMaxAndSum(max_queue_size, sum_queue_size, status.queue.queue_size);
calculateMaxAndSum(max_inserts_in_queue, sum_inserts_in_queue, status.queue.inserts_in_queue);
calculateMaxAndSum(max_merges_in_queue, sum_merges_in_queue, status.queue.merges_in_queue);
if (!status.is_readonly)
{
try
{
time_t absolute_delay = 0;
time_t relative_delay = 0;
table_replicated_merge_tree->getReplicaDelays(absolute_delay, relative_delay);
calculateMax(max_absolute_delay, absolute_delay);
calculateMax(max_relative_delay, relative_delay);
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__,
"Cannot get replica delay for table: " + backQuoteIfNeed(db.first) + "." + backQuoteIfNeed(iterator->name()));
}
}
}
}
}
new_values["ReplicasMaxQueueSize"] = { max_queue_size, "Maximum queue size (in the number of operations like get, merge) across Replicated tables." };
new_values["ReplicasMaxInsertsInQueue"] = { max_inserts_in_queue, "Maximum number of INSERT operations in the queue (still to be replicated) across Replicated tables." };
new_values["ReplicasMaxMergesInQueue"] = { max_merges_in_queue, "Maximum number of merge operations in the queue (still to be applied) across Replicated tables." };
new_values["ReplicasSumQueueSize"] = { sum_queue_size, "Sum queue size (in the number of operations like get, merge) across Replicated tables." };
new_values["ReplicasSumInsertsInQueue"] = { sum_inserts_in_queue, "Sum of INSERT operations in the queue (still to be replicated) across Replicated tables." };
new_values["ReplicasSumMergesInQueue"] = { sum_merges_in_queue, "Sum of merge operations in the queue (still to be applied) across Replicated tables." };
new_values["ReplicasMaxAbsoluteDelay"] = { max_absolute_delay, "Maximum difference in seconds between the most fresh replicated part and the most fresh data part still to be replicated, across Replicated tables. A very high value indicates a replica with no data." };
new_values["ReplicasMaxRelativeDelay"] = { max_relative_delay, "Maximum difference between the replica delay and the delay of the most up-to-date replica of the same table, across Replicated tables." };
new_values["MaxPartCountForPartition"] = { max_part_count_for_partition, "Maximum number of parts per partition across all partitions of all tables of MergeTree family. Values larger than 300 indicates misconfiguration, overload, or massive data loading." };
new_values["NumberOfDatabases"] = { number_of_databases, "Total number of databases on the server." };
new_values["NumberOfTables"] = { total_number_of_tables, "Total number of tables summed across the databases on the server, excluding the databases that cannot contain MergeTree tables."
" The excluded database engines are those who generate the set of tables on the fly, like `Lazy`, `MySQL`, `PostgreSQL`, `SQlite`."};
new_values["TotalBytesOfMergeTreeTables"] = { total_number_of_bytes, "Total amount of bytes (compressed, including data and indices) stored in all tables of MergeTree family." };
new_values["TotalRowsOfMergeTreeTables"] = { total_number_of_rows, "Total amount of rows (records) stored in all tables of MergeTree family." };
new_values["TotalPartsOfMergeTreeTables"] = { total_number_of_parts, "Total amount of data parts in all tables of MergeTree family."
" Numbers larger than 10 000 will negatively affect the server startup time and it may indicate unreasonable choice of the partition key." };
}
#if USE_NURAFT
{
auto keeper_dispatcher = getContext()->tryGetKeeperDispatcher();
if (keeper_dispatcher)
updateKeeperInformation(*keeper_dispatcher, new_values);
}
#endif
updateHeavyMetricsIfNeeded(current_time, update_time, new_values);
}
void ServerAsynchronousMetrics::logImpl(AsynchronousMetricValues & new_values)
{
/// Log the new metrics.
if (auto asynchronous_metric_log = getContext()->getAsynchronousMetricLog())
asynchronous_metric_log->addValues(new_values);
}
void ServerAsynchronousMetrics::updateDetachedPartsStats()
{
DetachedPartsStats current_values{};
for (const auto & db : DatabaseCatalog::instance().getDatabases())
{
if (!db.second->canContainMergeTreeTables())
continue;
for (auto iterator = db.second->getTablesIterator(getContext()); iterator->isValid(); iterator->next())
{
const auto & table = iterator->table();
if (!table)
continue;
if (MergeTreeData * table_merge_tree = dynamic_cast<MergeTreeData *>(table.get()))
{
for (const auto & detached_part: table_merge_tree->getDetachedParts())
{
if (!detached_part.valid_name)
continue;
if (detached_part.prefix.empty())
++current_values.detached_by_user;
++current_values.count;
}
}
}
}
detached_parts_stats = current_values;
}
void ServerAsynchronousMetrics::updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, AsynchronousMetricValues & new_values)
{
const auto time_after_previous_update = current_time - heavy_metric_previous_update_time;
const bool update_heavy_metric = time_after_previous_update >= heavy_metric_update_period || first_run;
if (update_heavy_metric)
{
heavy_metric_previous_update_time = update_time;
Stopwatch watch;
/// Test shows that listing 100000 entries consuming around 0.15 sec.
updateDetachedPartsStats();
watch.stop();
/// Normally heavy metrics don't delay the rest of the metrics calculation
/// otherwise log the warning message
auto log_level = std::make_pair(DB::LogsLevel::trace, Poco::Message::PRIO_TRACE);
if (watch.elapsedSeconds() > (update_period.count() / 2.))
log_level = std::make_pair(DB::LogsLevel::debug, Poco::Message::PRIO_DEBUG);
else if (watch.elapsedSeconds() > (update_period.count() / 4. * 3))
log_level = std::make_pair(DB::LogsLevel::warning, Poco::Message::PRIO_WARNING);
LOG_IMPL(log, log_level.first, log_level.second,
"Update heavy metrics. "
"Update period {} sec. "
"Update heavy metrics period {} sec. "
"Heavy metrics calculation elapsed: {} sec.",
update_period.count(),
heavy_metric_update_period.count(),
watch.elapsedSeconds());
}
new_values["NumberOfDetachedParts"] = { detached_parts_stats.count, "The total number of parts detached from MergeTree tables. A part can be detached by a user with the `ALTER TABLE DETACH` query or by the server itself it the part is broken, unexpected or unneeded. The server does not care about detached parts and they can be removed." };
new_values["NumberOfDetachedByUserParts"] = { detached_parts_stats.detached_by_user, "The total number of parts detached from MergeTree tables by users with the `ALTER TABLE DETACH` query (as opposed to unexpected, broken or ignored parts). The server does not care about detached parts and they can be removed." };
}
}

View File

@ -0,0 +1,37 @@
#pragma once
#include <Common/AsynchronousMetrics.h>
#include <Interpreters/Context_fwd.h>
namespace DB
{
class ServerAsynchronousMetrics : public AsynchronousMetrics, WithContext
{
public:
ServerAsynchronousMetrics(
ContextPtr global_context_,
int update_period_seconds,
int heavy_metrics_update_period_seconds,
const ProtocolServerMetricsFunc & protocol_server_metrics_func_);
private:
void updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) override;
void logImpl(AsynchronousMetricValues & new_values) override;
const Duration heavy_metric_update_period;
TimePoint heavy_metric_previous_update_time;
struct DetachedPartsStats
{
size_t count;
size_t detached_by_user;
};
DetachedPartsStats detached_parts_stats{};
void updateDetachedPartsStats();
void updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, AsynchronousMetricValues & new_values);
};
}

View File

@ -7,6 +7,7 @@
#include <Formats/NativeWriter.h>
#include <Formats/NativeReader.h>
#include <Core/ProtocolDefines.h>
#include <Disks/TemporaryFileInPath.h>
#include <Common/logger_useful.h>
@ -35,34 +36,31 @@ void TemporaryDataOnDiskScope::deltaAllocAndCheck(ssize_t compressed_delta, ssiz
size_t new_consumprion = stat.compressed_size + compressed_delta;
if (compressed_delta > 0 && limit && new_consumprion > limit)
throw Exception(ErrorCodes::TOO_MANY_ROWS_OR_BYTES, "Limit for temporary files size exceeded");
throw Exception(ErrorCodes::TOO_MANY_ROWS_OR_BYTES,
"Limit for temporary files size exceeded (would consume {} / {} bytes)", new_consumprion, limit);
stat.compressed_size += compressed_delta;
stat.uncompressed_size += uncompressed_delta;
}
TemporaryFileStream & TemporaryDataOnDisk::createStream(const Block & header, size_t max_file_size)
VolumePtr TemporaryDataOnDiskScope::getVolume() const
{
DiskPtr disk;
if (max_file_size > 0)
{
auto reservation = volume->reserve(max_file_size);
if (!reservation)
throw Exception("Not enough space on temporary disk", ErrorCodes::NOT_ENOUGH_SPACE);
disk = reservation->getDisk();
}
else
{
disk = volume->getDisk();
}
auto tmp_file = std::make_unique<TemporaryFileOnDisk>(disk, current_metric_scope);
std::lock_guard lock(mutex);
TemporaryFileStreamPtr & tmp_stream = streams.emplace_back(std::make_unique<TemporaryFileStream>(std::move(tmp_file), header, this));
return *tmp_stream;
if (!volume)
throw Exception("TemporaryDataOnDiskScope has no volume", ErrorCodes::LOGICAL_ERROR);
return volume;
}
TemporaryFileStream & TemporaryDataOnDisk::createStream(const Block & header, size_t max_file_size)
{
TemporaryFileStreamPtr tmp_stream;
if (cache)
tmp_stream = TemporaryFileStream::create(cache, header, max_file_size, this);
else
tmp_stream = TemporaryFileStream::create(volume, header, max_file_size, this);
std::lock_guard lock(mutex);
return *streams.emplace_back(std::move(tmp_stream));
}
std::vector<TemporaryFileStream *> TemporaryDataOnDisk::getStreams() const
{
@ -89,12 +87,13 @@ struct TemporaryFileStream::OutputWriter
{
}
void write(const Block & block)
size_t write(const Block & block)
{
if (finalized)
throw Exception("Cannot write to finalized stream", ErrorCodes::LOGICAL_ERROR);
out_writer.write(block);
size_t written_bytes = out_writer.write(block);
num_rows += block.rows();
return written_bytes;
}
void finalize()
@ -155,21 +154,68 @@ struct TemporaryFileStream::InputReader
NativeReader in_reader;
};
TemporaryFileStream::TemporaryFileStream(TemporaryFileOnDiskHolder file_, const Block & header_, TemporaryDataOnDisk * parent_)
TemporaryFileStreamPtr TemporaryFileStream::create(const VolumePtr & volume, const Block & header, size_t max_file_size, TemporaryDataOnDisk * parent_)
{
if (!volume)
throw Exception("TemporaryDataOnDiskScope has no volume", ErrorCodes::LOGICAL_ERROR);
DiskPtr disk;
if (max_file_size > 0)
{
auto reservation = volume->reserve(max_file_size);
if (!reservation)
throw Exception("Not enough space on temporary disk", ErrorCodes::NOT_ENOUGH_SPACE);
disk = reservation->getDisk();
}
else
{
disk = volume->getDisk();
}
auto tmp_file = std::make_unique<TemporaryFileOnDisk>(disk, parent_->getMetricScope());
return std::make_unique<TemporaryFileStream>(std::move(tmp_file), header, /* cache_placeholder */ nullptr, /* parent */ parent_);
}
TemporaryFileStreamPtr TemporaryFileStream::create(FileCache * cache, const Block & header, size_t max_file_size, TemporaryDataOnDisk * parent_)
{
auto tmp_file = std::make_unique<TemporaryFileInPath>(fs::path(cache->getBasePath()) / "tmp");
auto cache_placeholder = std::make_unique<FileCachePlaceholder>(cache, tmp_file->getPath());
cache_placeholder->reserveCapacity(max_file_size);
return std::make_unique<TemporaryFileStream>(std::move(tmp_file), header, std::move(cache_placeholder), parent_);
}
TemporaryFileStream::TemporaryFileStream(
TemporaryFileHolder file_,
const Block & header_,
std::unique_ptr<ISpacePlaceholder> space_holder_,
TemporaryDataOnDisk * parent_)
: parent(parent_)
, header(header_)
, file(std::move(file_))
, space_holder(std::move(space_holder_))
, out_writer(std::make_unique<OutputWriter>(file->getPath(), header))
{
}
void TemporaryFileStream::write(const Block & block)
size_t TemporaryFileStream::write(const Block & block)
{
if (!out_writer)
throw Exception("Writing has been finished", ErrorCodes::LOGICAL_ERROR);
size_t block_size_in_memory = block.bytes();
if (space_holder)
space_holder->reserveCapacity(block_size_in_memory);
updateAllocAndCheck();
out_writer->write(block);
size_t bytes_written = out_writer->write(block);
if (space_holder)
space_holder->setUsed(bytes_written);
return bytes_written;
}
TemporaryFileStream::Stat TemporaryFileStream::finishWriting()

View File

@ -6,6 +6,7 @@
#include <Disks/TemporaryFileOnDisk.h>
#include <Disks/IVolume.h>
#include <Common/CurrentMetrics.h>
#include <Disks/IO/FileCachePlaceholder.h>
namespace CurrentMetrics
@ -40,23 +41,25 @@ public:
std::atomic<size_t> uncompressed_size;
};
explicit TemporaryDataOnDiskScope(VolumePtr volume_, size_t limit_)
: volume(std::move(volume_)), limit(limit_)
explicit TemporaryDataOnDiskScope(VolumePtr volume_, FileCache * cache_, size_t limit_)
: volume(std::move(volume_)), cache(cache_), limit(limit_)
{}
explicit TemporaryDataOnDiskScope(TemporaryDataOnDiskScopePtr parent_, size_t limit_)
: parent(std::move(parent_)), volume(parent->volume), limit(limit_)
: parent(std::move(parent_)), volume(parent->volume), cache(parent->cache), limit(limit_)
{}
/// TODO: remove
/// Refactor all code that uses volume directly to use TemporaryDataOnDisk.
VolumePtr getVolume() const { return volume; }
VolumePtr getVolume() const;
protected:
void deltaAllocAndCheck(ssize_t compressed_delta, ssize_t uncompressed_delta);
TemporaryDataOnDiskScopePtr parent = nullptr;
VolumePtr volume;
FileCache * cache = nullptr;
StatAtomic stat;
size_t limit = 0;
@ -91,6 +94,7 @@ public:
bool empty() const;
const StatAtomic & getStat() const { return stat; }
CurrentMetrics::Value getMetricScope() const { return current_metric_scope; }
private:
mutable std::mutex mutex;
@ -116,9 +120,14 @@ public:
size_t num_rows = 0;
};
TemporaryFileStream(TemporaryFileOnDiskHolder file_, const Block & header_, TemporaryDataOnDisk * parent_);
static TemporaryFileStreamPtr create(const VolumePtr & volume, const Block & header, size_t max_file_size, TemporaryDataOnDisk * parent_);
static TemporaryFileStreamPtr create(FileCache * cache, const Block & header, size_t max_file_size, TemporaryDataOnDisk * parent_);
TemporaryFileStream(TemporaryFileHolder file_, const Block & header_, std::unique_ptr<ISpacePlaceholder> space_holder, TemporaryDataOnDisk * parent_);
/// Returns number of written bytes
size_t write(const Block & block);
void write(const Block & block);
Stat finishWriting();
bool isWriteFinished() const;
@ -142,7 +151,8 @@ private:
Block header;
TemporaryFileOnDiskHolder file;
TemporaryFileHolder file;
std::unique_ptr<ISpacePlaceholder> space_holder;
Stat stat;

View File

@ -6,6 +6,7 @@
#include <Common/CurrentThread.h>
#include <Common/filesystemHelpers.h>
#include <Interpreters/Cache/FileCacheSettings.h>
#include <Interpreters/TemporaryDataOnDisk.h>
#include <Common/tests/gtest_global_context.h>
#include <Common/SipHash.h>
#include <Common/hex.h>
@ -14,11 +15,14 @@
#include <IO/WriteHelpers.h>
#include <filesystem>
#include <thread>
#include <DataTypes/DataTypesNumber.h>
#include <Disks/IO/CachedOnDiskWriteBufferFromFile.h>
namespace fs = std::filesystem;
using namespace DB;
fs::path caches_dir = fs::current_path() / "lru_cache_test";
String cache_base_path = caches_dir / "cache1" / "";
static constexpr auto TEST_LOG_LEVEL = "debug";
void assertRange(
[[maybe_unused]] size_t assert_n, DB::FileSegmentPtr file_segment,
@ -53,7 +57,7 @@ String getFileSegmentPath(const String & base_path, const DB::FileCache::Key & k
return fs::path(base_path) / key_str.substr(0, 3) / key_str / DB::toString(offset);
}
void download(DB::FileSegmentPtr file_segment)
void download(const std::string & cache_base_path, DB::FileSegmentPtr file_segment)
{
const auto & key = file_segment->key();
size_t size = file_segment->range().size();
@ -67,30 +71,57 @@ void download(DB::FileSegmentPtr file_segment)
file_segment->write(data.data(), size, file_segment->getCurrentWriteOffset());
}
void prepareAndDownload(DB::FileSegmentPtr file_segment)
void prepareAndDownload(const std::string & cache_base_path, DB::FileSegmentPtr file_segment)
{
// std::cerr << "Reserving: " << file_segment->range().size() << " for: " << file_segment->range().toString() << "\n";
ASSERT_TRUE(file_segment->reserve(file_segment->range().size()));
download(file_segment);
download(cache_base_path, file_segment);
}
void complete(const DB::FileSegmentsHolder & holder)
void complete(const std::string & cache_base_path, const DB::FileSegmentsHolder & holder)
{
for (const auto & file_segment : holder.file_segments)
{
ASSERT_TRUE(file_segment->getOrSetDownloader() == DB::FileSegment::getCallerId());
prepareAndDownload(file_segment);
prepareAndDownload(cache_base_path, file_segment);
file_segment->completeWithState(DB::FileSegment::State::DOWNLOADED);
}
}
TEST(FileCache, get)
class FileCacheTest : public ::testing::Test
{
if (fs::exists(cache_base_path))
fs::remove_all(cache_base_path);
fs::create_directories(cache_base_path);
public:
static void setupLogs(const std::string & level)
{
Poco::AutoPtr<Poco::ConsoleChannel> channel(new Poco::ConsoleChannel(std::cerr));
Poco::Logger::root().setChannel(channel);
Poco::Logger::root().setLevel(level);
}
void SetUp() override
{
if(const char * test_log_level = std::getenv("TEST_LOG_LEVEL")) // NOLINT(concurrency-mt-unsafe)
setupLogs(test_log_level);
else
setupLogs(TEST_LOG_LEVEL);
if (fs::exists(cache_base_path))
fs::remove_all(cache_base_path);
fs::create_directories(cache_base_path);
}
void TearDown() override
{
if (fs::exists(cache_base_path))
fs::remove_all(cache_base_path);
}
fs::path caches_dir = fs::current_path() / "lru_cache_test";
std::string cache_base_path = caches_dir / "cache1" / "";
};
TEST_F(FileCacheTest, get)
{
DB::ThreadStatus thread_status;
/// To work with cache need query_id and query context.
@ -126,7 +157,7 @@ TEST(FileCache, get)
ASSERT_TRUE(segments[0]->reserve(segments[0]->range().size()));
assertRange(2, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADING);
download(segments[0]);
download(cache_base_path, segments[0]);
segments[0]->completeWithState(DB::FileSegment::State::DOWNLOADED);
assertRange(3, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED);
}
@ -147,7 +178,7 @@ TEST(FileCache, get)
assertRange(5, segments[1], DB::FileSegment::Range(10, 14), DB::FileSegment::State::EMPTY);
ASSERT_TRUE(segments[1]->getOrSetDownloader() == DB::FileSegment::getCallerId());
prepareAndDownload(segments[1]);
prepareAndDownload(cache_base_path, segments[1]);
segments[1]->completeWithState(DB::FileSegment::State::DOWNLOADED);
assertRange(6, segments[1], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED);
}
@ -180,8 +211,8 @@ TEST(FileCache, get)
assertRange(10, segments[0], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED);
}
complete(cache.getOrSet(key, 17, 4, {})); /// Get [17, 20]
complete(cache.getOrSet(key, 24, 3, {})); /// Get [24, 26]
complete(cache_base_path, cache.getOrSet(key, 17, 4, {})); /// Get [17, 20]
complete(cache_base_path, cache.getOrSet(key, 24, 3, {})); /// Get [24, 26]
/// completeWithState(cache.getOrSet(key, 27, 1, false)); /// Get [27, 27]
/// Current cache: [__________][_____] [____] [___][]
@ -203,7 +234,7 @@ TEST(FileCache, get)
assertRange(13, segments[2], DB::FileSegment::Range(15, 16), DB::FileSegment::State::EMPTY);
ASSERT_TRUE(segments[2]->getOrSetDownloader() == DB::FileSegment::getCallerId());
prepareAndDownload(segments[2]);
prepareAndDownload(cache_base_path, segments[2]);
segments[2]->completeWithState(DB::FileSegment::State::DOWNLOADED);
@ -244,7 +275,7 @@ TEST(FileCache, get)
assertRange(21, segments[3], DB::FileSegment::Range(21, 21), DB::FileSegment::State::EMPTY);
ASSERT_TRUE(segments[3]->getOrSetDownloader() == DB::FileSegment::getCallerId());
prepareAndDownload(segments[3]);
prepareAndDownload(cache_base_path, segments[3]);
segments[3]->completeWithState(DB::FileSegment::State::DOWNLOADED);
ASSERT_TRUE(segments[3]->state() == DB::FileSegment::State::DOWNLOADED);
@ -267,8 +298,8 @@ TEST(FileCache, get)
ASSERT_TRUE(segments[0]->getOrSetDownloader() == DB::FileSegment::getCallerId());
ASSERT_TRUE(segments[2]->getOrSetDownloader() == DB::FileSegment::getCallerId());
prepareAndDownload(segments[0]);
prepareAndDownload(segments[2]);
prepareAndDownload(cache_base_path, segments[0]);
prepareAndDownload(cache_base_path, segments[2]);
segments[0]->completeWithState(DB::FileSegment::State::DOWNLOADED);
segments[2]->completeWithState(DB::FileSegment::State::DOWNLOADED);
}
@ -290,8 +321,8 @@ TEST(FileCache, get)
ASSERT_TRUE(s5[0]->getOrSetDownloader() == DB::FileSegment::getCallerId());
ASSERT_TRUE(s1[0]->getOrSetDownloader() == DB::FileSegment::getCallerId());
prepareAndDownload(s5[0]);
prepareAndDownload(s1[0]);
prepareAndDownload(cache_base_path, s5[0]);
prepareAndDownload(cache_base_path, s1[0]);
s5[0]->completeWithState(DB::FileSegment::State::DOWNLOADED);
s1[0]->completeWithState(DB::FileSegment::State::DOWNLOADED);
@ -394,7 +425,7 @@ TEST(FileCache, get)
cv.wait(lock, [&]{ return lets_start_download; });
}
prepareAndDownload(segments[2]);
prepareAndDownload(cache_base_path, segments[2]);
segments[2]->completeWithState(DB::FileSegment::State::DOWNLOADED);
ASSERT_TRUE(segments[2]->state() == DB::FileSegment::State::DOWNLOADED);
@ -459,7 +490,7 @@ TEST(FileCache, get)
ASSERT_TRUE(segments_2[1]->state() == DB::FileSegment::State::PARTIALLY_DOWNLOADED);
ASSERT_TRUE(segments_2[1]->getOrSetDownloader() == DB::FileSegment::getCallerId());
prepareAndDownload(segments_2[1]);
prepareAndDownload(cache_base_path, segments_2[1]);
segments_2[1]->completeWithState(DB::FileSegment::State::DOWNLOADED);
});
@ -517,3 +548,141 @@ TEST(FileCache, get)
}
}
TEST_F(FileCacheTest, rangeWriter)
{
DB::FileCacheSettings settings;
settings.max_size = 25;
settings.max_elements = 5;
settings.max_file_segment_size = 10;
DB::FileCache cache(cache_base_path, settings);
cache.initialize();
auto key = cache.hash("key1");
DB::FileSegmentRangeWriter writer(&cache, key, nullptr, "", "key1");
std::string data(100, '\xf0');
size_t total_written = 0;
for (const size_t size : {3, 5, 8, 1, 1, 3})
{
total_written += size;
ASSERT_EQ(writer.tryWrite(data.data(), size, writer.currentOffset()), size);
}
ASSERT_LT(total_written, settings.max_size);
size_t offset_before_unsuccessful_write = writer.currentOffset();
size_t space_left = settings.max_size - total_written;
ASSERT_EQ(writer.tryWrite(data.data(), space_left + 1, writer.currentOffset()), 0);
ASSERT_EQ(writer.currentOffset(), offset_before_unsuccessful_write);
ASSERT_EQ(writer.tryWrite(data.data(), space_left, writer.currentOffset()), space_left);
writer.finalize();
}
static Block generateBlock(size_t size = 0)
{
Block block;
ColumnWithTypeAndName column;
column.name = "x";
column.type = std::make_shared<DataTypeUInt64>();
{
MutableColumnPtr mut_col = column.type->createColumn();
for (size_t i = 0; i < size; ++i)
mut_col->insert(i);
column.column = std::move(mut_col);
}
block.insert(column);
LOG_DEBUG(&Poco::Logger::get("FileCacheTest"), "generated block {} bytes", block.bytes());
return block;
}
static size_t readAllTemporaryData(TemporaryFileStream & stream)
{
Block block;
size_t read_rows = 0;
do
{
block = stream.read();
read_rows += block.rows();
} while (block);
return read_rows;
}
TEST_F(FileCacheTest, temporaryData)
{
DB::FileCacheSettings settings;
settings.max_size = 10240;
settings.max_file_segment_size = 1024;
DB::FileCache file_cache(cache_base_path, settings);
file_cache.initialize();
auto tmp_data_scope = std::make_shared<TemporaryDataOnDiskScope>(nullptr, &file_cache, 0);
auto some_data_holder = file_cache.getOrSet(file_cache.hash("some_data"), 0, 1024 * 5, CreateFileSegmentSettings{});
{
auto segments = fromHolder(some_data_holder);
ASSERT_EQ(segments.size(), 5);
for (auto & segment : segments)
{
ASSERT_TRUE(segment->getOrSetDownloader() == DB::FileSegment::getCallerId());
ASSERT_TRUE(segment->reserve(segment->range().size()));
download(cache_base_path, segment);
segment->completeWithState(DB::FileSegment::State::DOWNLOADED);
}
}
size_t size_used_before_temporary_data = file_cache.getUsedCacheSize();
size_t segments_used_before_temporary_data = file_cache.getFileSegmentsNum();
ASSERT_GT(size_used_before_temporary_data, 0);
ASSERT_GT(segments_used_before_temporary_data, 0);
size_t size_used_with_temporary_data;
size_t segments_used_with_temporary_data;
{
auto tmp_data = std::make_unique<TemporaryDataOnDisk>(tmp_data_scope);
auto & stream = tmp_data->createStream(generateBlock());
ASSERT_GT(stream.write(generateBlock(100)), 0);
EXPECT_GT(file_cache.getUsedCacheSize(), 0);
EXPECT_GT(file_cache.getFileSegmentsNum(), 0);
size_t used_size_before_attempt = file_cache.getUsedCacheSize();
/// data can't be evicted because it is still held by `some_data_holder`
ASSERT_THROW(stream.write(generateBlock(1000)), DB::Exception);
ASSERT_EQ(file_cache.getUsedCacheSize(), used_size_before_attempt);
some_data_holder.reset();
stream.write(generateBlock(1011));
auto stat = stream.finishWriting();
EXPECT_EQ(stat.num_rows, 1111);
EXPECT_EQ(readAllTemporaryData(stream), 1111);
size_used_with_temporary_data = file_cache.getUsedCacheSize();
segments_used_with_temporary_data = file_cache.getFileSegmentsNum();
EXPECT_GT(size_used_with_temporary_data, 0);
EXPECT_GT(segments_used_with_temporary_data, 0);
}
/// All temp data should be evicted after removing temporary files
EXPECT_LE(file_cache.getUsedCacheSize(), size_used_with_temporary_data);
EXPECT_LE(file_cache.getFileSegmentsNum(), segments_used_with_temporary_data);
/// Some segments reserved by `some_data_holder` was eviced by temporary data
EXPECT_LE(file_cache.getUsedCacheSize(), size_used_before_temporary_data);
EXPECT_LE(file_cache.getFileSegmentsNum(), segments_used_before_temporary_data);
}

View File

@ -0,0 +1,24 @@
#pragma once
#include <Poco/Timespan.h>
namespace DB
{
struct IHTTPContext
{
virtual uint64_t getMaxHstsAge() const = 0;
virtual uint64_t getMaxUriSize() const = 0;
virtual uint64_t getMaxFields() const = 0;
virtual uint64_t getMaxFieldNameSize() const = 0;
virtual uint64_t getMaxFieldValueSize() const = 0;
virtual uint64_t getMaxChunkSize() const = 0;
virtual Poco::Timespan getReceiveTimeout() const = 0;
virtual Poco::Timespan getSendTimeout() const = 0;
virtual ~IHTTPContext() = default;
};
using HTTPContextPtr = std::shared_ptr<IHTTPContext>;
}

View File

@ -6,7 +6,7 @@
namespace DB
{
HTTPServer::HTTPServer(
ContextPtr context,
HTTPContextPtr context,
HTTPRequestHandlerFactoryPtr factory_,
Poco::ThreadPool & thread_pool,
Poco::Net::ServerSocket & socket_,

View File

@ -1,6 +1,7 @@
#pragma once
#include <Server/HTTP/HTTPRequestHandlerFactory.h>
#include <Server/HTTP/HTTPContext.h>
#include <Server/TCPServer.h>
#include <Poco/Net/HTTPServerParams.h>
@ -11,13 +12,11 @@
namespace DB
{
class Context;
class HTTPServer : public TCPServer
{
public:
explicit HTTPServer(
ContextPtr context,
HTTPContextPtr context,
HTTPRequestHandlerFactoryPtr factory,
Poco::ThreadPool & thread_pool,
Poco::Net::ServerSocket & socket,

View File

@ -7,12 +7,12 @@ namespace DB
{
HTTPServerConnection::HTTPServerConnection(
ContextPtr context_,
HTTPContextPtr context_,
TCPServer & tcp_server_,
const Poco::Net::StreamSocket & socket,
Poco::Net::HTTPServerParams::Ptr params_,
HTTPRequestHandlerFactoryPtr factory_)
: TCPServerConnection(socket), context(Context::createCopy(context_)), tcp_server(tcp_server_), params(params_), factory(factory_), stopped(false)
: TCPServerConnection(socket), context(std::move(context_)), tcp_server(tcp_server_), params(params_), factory(factory_), stopped(false)
{
poco_check_ptr(factory);
}
@ -36,7 +36,7 @@ void HTTPServerConnection::run()
if (request.isSecure())
{
size_t hsts_max_age = context->getSettingsRef().hsts_max_age.value;
size_t hsts_max_age = context->getMaxHstsAge();
if (hsts_max_age > 0)
response.add("Strict-Transport-Security", "max-age=" + std::to_string(hsts_max_age));

View File

@ -1,7 +1,7 @@
#pragma once
#include <Interpreters/Context.h>
#include <Server/HTTP/HTTPRequestHandlerFactory.h>
#include <Server/HTTP/HTTPContext.h>
#include <Poco/Net/HTTPServerParams.h>
#include <Poco/Net/HTTPServerSession.h>
@ -15,7 +15,7 @@ class HTTPServerConnection : public Poco::Net::TCPServerConnection
{
public:
HTTPServerConnection(
ContextPtr context,
HTTPContextPtr context,
TCPServer & tcp_server,
const Poco::Net::StreamSocket & socket,
Poco::Net::HTTPServerParams::Ptr params,
@ -27,7 +27,7 @@ protected:
static void sendErrorResponse(Poco::Net::HTTPServerSession & session, Poco::Net::HTTPResponse::HTTPStatus status);
private:
ContextPtr context;
HTTPContextPtr context;
TCPServer & tcp_server;
Poco::Net::HTTPServerParams::Ptr params;
HTTPRequestHandlerFactoryPtr factory;

View File

@ -5,8 +5,8 @@
namespace DB
{
HTTPServerConnectionFactory::HTTPServerConnectionFactory(
ContextPtr context_, Poco::Net::HTTPServerParams::Ptr params_, HTTPRequestHandlerFactoryPtr factory_)
: context(Context::createCopy(context_)), params(params_), factory(factory_)
HTTPContextPtr context_, Poco::Net::HTTPServerParams::Ptr params_, HTTPRequestHandlerFactoryPtr factory_)
: context(std::move(context_)), params(params_), factory(factory_)
{
poco_check_ptr(factory);
}

View File

@ -1,7 +1,7 @@
#pragma once
#include <Interpreters/Context.h>
#include <Server/HTTP/HTTPRequestHandlerFactory.h>
#include <Server/HTTP/HTTPContext.h>
#include <Server/TCPServerConnectionFactory.h>
#include <Poco/Net/HTTPServerParams.h>
@ -12,12 +12,12 @@ namespace DB
class HTTPServerConnectionFactory : public TCPServerConnectionFactory
{
public:
HTTPServerConnectionFactory(ContextPtr context, Poco::Net::HTTPServerParams::Ptr params, HTTPRequestHandlerFactoryPtr factory);
HTTPServerConnectionFactory(HTTPContextPtr context, Poco::Net::HTTPServerParams::Ptr params, HTTPRequestHandlerFactoryPtr factory);
Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket, TCPServer & tcp_server) override;
private:
ContextPtr context;
HTTPContextPtr context;
Poco::Net::HTTPServerParams::Ptr params;
HTTPRequestHandlerFactoryPtr factory;
};

View File

@ -1,6 +1,5 @@
#include <Server/HTTP/HTTPServerRequest.h>
#include <Interpreters/Context.h>
#include <IO/EmptyReadBuffer.h>
#include <IO/HTTPChunkedReadBuffer.h>
#include <IO/LimitReadBuffer.h>
@ -21,11 +20,11 @@
namespace DB
{
HTTPServerRequest::HTTPServerRequest(ContextPtr context, HTTPServerResponse & response, Poco::Net::HTTPServerSession & session)
: max_uri_size(context->getSettingsRef().http_max_uri_size)
, max_fields_number(context->getSettingsRef().http_max_fields)
, max_field_name_size(context->getSettingsRef().http_max_field_name_size)
, max_field_value_size(context->getSettingsRef().http_max_field_value_size)
HTTPServerRequest::HTTPServerRequest(HTTPContextPtr context, HTTPServerResponse & response, Poco::Net::HTTPServerSession & session)
: max_uri_size(context->getMaxUriSize())
, max_fields_number(context->getMaxFields())
, max_field_name_size(context->getMaxFieldNameSize())
, max_field_value_size(context->getMaxFieldValueSize())
{
response.attachRequest(this);
@ -34,8 +33,8 @@ HTTPServerRequest::HTTPServerRequest(ContextPtr context, HTTPServerResponse & re
server_address = session.serverAddress();
secure = session.socket().secure();
auto receive_timeout = context->getSettingsRef().http_receive_timeout;
auto send_timeout = context->getSettingsRef().http_send_timeout;
auto receive_timeout = context->getReceiveTimeout();
auto send_timeout = context->getSendTimeout();
session.socket().setReceiveTimeout(receive_timeout);
session.socket().setSendTimeout(send_timeout);
@ -46,7 +45,7 @@ HTTPServerRequest::HTTPServerRequest(ContextPtr context, HTTPServerResponse & re
readRequest(*in); /// Try parse according to RFC7230
if (getChunkedTransferEncoding())
stream = std::make_unique<HTTPChunkedReadBuffer>(std::move(in), context->getSettingsRef().http_max_chunk_size);
stream = std::make_unique<HTTPChunkedReadBuffer>(std::move(in), context->getMaxChunkSize());
else if (hasContentLength())
stream = std::make_unique<LimitReadBuffer>(std::move(in), getContentLength(), false);
else if (getMethod() != HTTPRequest::HTTP_GET && getMethod() != HTTPRequest::HTTP_HEAD && getMethod() != HTTPRequest::HTTP_DELETE)

View File

@ -3,6 +3,7 @@
#include <Interpreters/Context_fwd.h>
#include <IO/ReadBuffer.h>
#include <Server/HTTP/HTTPRequest.h>
#include <Server/HTTP/HTTPContext.h>
#include "config.h"
#include <Poco/Net/HTTPServerSession.h>
@ -18,7 +19,7 @@ class ReadBufferFromPocoSocket;
class HTTPServerRequest : public HTTPRequest
{
public:
HTTPServerRequest(ContextPtr context, HTTPServerResponse & response, Poco::Net::HTTPServerSession & session);
HTTPServerRequest(HTTPContextPtr context, HTTPServerResponse & response, Poco::Net::HTTPServerSession & session);
/// FIXME: it's a little bit inconvenient interface. The rationale is that all other ReadBuffer's wrap each other
/// via unique_ptr - but we can't inherit HTTPServerRequest from ReadBuffer and pass it around,

View File

@ -3,7 +3,6 @@
#include <Server/HTTP/HTTPRequestHandler.h>
#include <Server/IServer.h>
#include <Access/Credentials.h>
#include <Interpreters/Context.h>
#include <Poco/Util/AbstractConfiguration.h>
@ -33,35 +32,6 @@ static void addDefaultHandlersFactory(
const Poco::Util::AbstractConfiguration & config,
AsynchronousMetrics & async_metrics);
HTTPRequestHandlerFactoryMain::HTTPRequestHandlerFactoryMain(const std::string & name_)
: log(&Poco::Logger::get(name_)), name(name_)
{
}
std::unique_ptr<HTTPRequestHandler> HTTPRequestHandlerFactoryMain::createRequestHandler(const HTTPServerRequest & request)
{
LOG_TRACE(log, "HTTP Request for {}. Method: {}, Address: {}, User-Agent: {}{}, Content Type: {}, Transfer Encoding: {}, X-Forwarded-For: {}",
name, request.getMethod(), request.clientAddress().toString(), request.get("User-Agent", "(none)"),
(request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")),
request.getContentType(), request.getTransferEncoding(), request.get("X-Forwarded-For", "(none)"));
for (auto & handler_factory : child_factories)
{
auto handler = handler_factory->createRequestHandler(request);
if (handler)
return handler;
}
if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET
|| request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD
|| request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST)
{
return std::unique_ptr<HTTPRequestHandler>(new NotFoundHandler);
}
return nullptr;
}
static inline auto createHandlersFactoryFromConfig(
IServer & server,
const Poco::Util::AbstractConfiguration & config,
@ -144,15 +114,7 @@ HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server, const Poco::
else if (name == "InterserverIOHTTPHandler-factory" || name == "InterserverIOHTTPSHandler-factory")
return createInterserverHTTPHandlerFactory(server, name);
else if (name == "PrometheusHandler-factory")
{
auto factory = std::make_shared<HTTPRequestHandlerFactoryMain>(name);
auto handler = std::make_shared<HandlingRuleHTTPHandlerFactory<PrometheusRequestHandler>>(
server, PrometheusMetricsWriter(config, "prometheus", async_metrics));
handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics"));
handler->allowGetAndHeadRequest();
factory->addHandler(handler);
return factory;
}
return createPrometheusMainHandlerFactory(server, config, async_metrics, name);
throw Exception("LOGICAL ERROR: Unknown HTTP handler factory name.", ErrorCodes::LOGICAL_ERROR);
}

View File

@ -1,9 +1,10 @@
#pragma once
#include <Interpreters/AsynchronousMetrics.h>
#include <Common/AsynchronousMetrics.h>
#include <Server/HTTP/HTMLForm.h>
#include <Server/HTTP/HTTPRequestHandlerFactory.h>
#include <Server/HTTPHandlerRequestFilter.h>
#include <Server/HTTPRequestHandlerFactoryMain.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/logger_useful.h>
@ -19,23 +20,6 @@ namespace ErrorCodes
class IServer;
/// Handle request using child handlers
class HTTPRequestHandlerFactoryMain : public HTTPRequestHandlerFactory
{
public:
explicit HTTPRequestHandlerFactoryMain(const std::string & name_);
void addHandler(HTTPRequestHandlerFactoryPtr child_factory) { child_factories.emplace_back(child_factory); }
std::unique_ptr<HTTPRequestHandler> createRequestHandler(const HTTPServerRequest & request) override;
private:
Poco::Logger * log;
std::string name;
std::vector<HTTPRequestHandlerFactoryPtr> child_factories;
};
template <typename TEndpoint>
class HandlingRuleHTTPHandlerFactory : public HTTPRequestHandlerFactory
{
@ -148,6 +132,12 @@ createPrometheusHandlerFactory(IServer & server,
AsynchronousMetrics & async_metrics,
const std::string & config_prefix);
HTTPRequestHandlerFactoryPtr
createPrometheusMainHandlerFactory(IServer & server,
const Poco::Util::AbstractConfiguration & config,
AsynchronousMetrics & async_metrics,
const std::string & name);
/// @param server - used in handlers to check IServer::isCancelled()
/// @param config - not the same as server.config(), since it can be newer
/// @param async_metrics - used for prometheus (in case of prometheus.asynchronous_metrics=true)

View File

@ -0,0 +1,38 @@
#include <Server/HTTPRequestHandlerFactoryMain.h>
#include <Server/NotFoundHandler.h>
#include <Common/logger_useful.h>
namespace DB
{
HTTPRequestHandlerFactoryMain::HTTPRequestHandlerFactoryMain(const std::string & name_)
: log(&Poco::Logger::get(name_)), name(name_)
{
}
std::unique_ptr<HTTPRequestHandler> HTTPRequestHandlerFactoryMain::createRequestHandler(const HTTPServerRequest & request)
{
LOG_TRACE(log, "HTTP Request for {}. Method: {}, Address: {}, User-Agent: {}{}, Content Type: {}, Transfer Encoding: {}, X-Forwarded-For: {}",
name, request.getMethod(), request.clientAddress().toString(), request.get("User-Agent", "(none)"),
(request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")),
request.getContentType(), request.getTransferEncoding(), request.get("X-Forwarded-For", "(none)"));
for (auto & handler_factory : child_factories)
{
auto handler = handler_factory->createRequestHandler(request);
if (handler)
return handler;
}
if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET
|| request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD
|| request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST)
{
return std::unique_ptr<HTTPRequestHandler>(new NotFoundHandler);
}
return nullptr;
}
}

View File

@ -0,0 +1,27 @@
#pragma once
#include <Server/HTTP/HTTPRequestHandlerFactory.h>
#include <vector>
namespace DB
{
/// Handle request using child handlers
class HTTPRequestHandlerFactoryMain : public HTTPRequestHandlerFactory
{
public:
explicit HTTPRequestHandlerFactoryMain(const std::string & name_);
void addHandler(HTTPRequestHandlerFactoryPtr child_factory) { child_factories.emplace_back(child_factory); }
std::unique_ptr<HTTPRequestHandler> createRequestHandler(const HTTPServerRequest & request) override;
private:
Poco::Logger * log;
std::string name;
std::vector<HTTPRequestHandlerFactoryPtr> child_factories;
};
}

View File

@ -2,7 +2,7 @@
#include <string>
#include <Interpreters/AsynchronousMetrics.h>
#include <Common/AsynchronousMetrics.h>
#include <IO/WriteBuffer.h>
#include <Poco/Util/AbstractConfiguration.h>

View File

@ -53,4 +53,19 @@ createPrometheusHandlerFactory(IServer & server,
return factory;
}
HTTPRequestHandlerFactoryPtr
createPrometheusMainHandlerFactory(IServer & server,
const Poco::Util::AbstractConfiguration & config,
AsynchronousMetrics & async_metrics,
const std::string & name)
{
auto factory = std::make_shared<HTTPRequestHandlerFactoryMain>(name);
auto handler = std::make_shared<HandlingRuleHTTPHandlerFactory<PrometheusRequestHandler>>(
server, PrometheusMetricsWriter(config, "prometheus", async_metrics));
handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics"));
handler->allowGetAndHeadRequest();
factory->addHandler(handler);
return factory;
}
}

View File

@ -101,6 +101,7 @@ public:
bool isValid() const override { return it->isValid(); }
bool isFile() const override { return isValid() && disk->isFile(it->path()); }
std::string name() const override { return it->name(); }
std::string path() const override { return it->path(); }
private:
DiskPtr disk;
@ -259,9 +260,17 @@ void DataPartStorageOnDisk::remove(
std::string proj_dir_name = projection.name + proj_suffix;
projection_directories.emplace(proj_dir_name);
NameSet files_not_to_remove_for_projection;
for (const auto & file_name : can_remove_description->files_not_to_remove)
{
if (file_name.starts_with(proj_dir_name))
files_not_to_remove_for_projection.emplace(fs::path(file_name).filename());
}
LOG_DEBUG(log, "Will not remove files [{}] for projection {}", fmt::join(files_not_to_remove_for_projection, ", "), projection.name);
clearDirectory(
fs::path(to) / proj_dir_name,
can_remove_description->can_remove_anything, can_remove_description->files_not_to_remove, projection.checksums, {}, is_temp, state, log, true);
can_remove_description->can_remove_anything, files_not_to_remove_for_projection, projection.checksums, {}, is_temp, state, log, true);
}
/// It is possible that we are removing the part which have a written but not loaded projection.

View File

@ -39,6 +39,9 @@ public:
/// Name of the file that the iterator currently points to.
virtual std::string name() const = 0;
/// Path of the file that the iterator currently points to.
virtual std::string path() const = 0;
virtual ~IDataPartStorageIterator() = default;
};

View File

@ -192,7 +192,7 @@ private:
bool force_ttl{false};
CompressionCodecPtr compression_codec{nullptr};
size_t sum_input_rows_upper_bound{0};
std::unique_ptr<TemporaryFile> rows_sources_file{nullptr};
std::unique_ptr<PocoTemporaryFile> rows_sources_file{nullptr};
std::unique_ptr<WriteBufferFromFileBase> rows_sources_uncompressed_write_buf{nullptr};
std::unique_ptr<WriteBuffer> rows_sources_write_buf{nullptr};
std::optional<ColumnSizeEstimator> column_sizes{};
@ -257,7 +257,7 @@ private:
/// Begin dependencies from previous stage
std::unique_ptr<WriteBuffer> rows_sources_write_buf{nullptr};
std::unique_ptr<WriteBufferFromFileBase> rows_sources_uncompressed_write_buf{nullptr};
std::unique_ptr<TemporaryFile> rows_sources_file;
std::unique_ptr<PocoTemporaryFile> rows_sources_file;
std::optional<ColumnSizeEstimator> column_sizes;
CompressionCodecPtr compression_codec;
DiskPtr tmp_disk{nullptr};

View File

@ -6437,7 +6437,25 @@ std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> MergeTreeData::cloneAn
if (!files_to_copy_instead_of_hardlinks.contains(it->name())
&& it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME
&& it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME)
{
hardlinked_files->hardlinks_from_source_part.insert(it->name());
}
}
auto projections = src_part->getProjectionParts();
for (const auto & [name, projection_part] : projections)
{
const auto & projection_storage = projection_part->getDataPartStorage();
for (auto it = projection_storage.iterate(); it->isValid(); it->next())
{
auto file_name_with_projection_prefix = fs::path(projection_storage.getPartDirectory()) / it->name();
if (!files_to_copy_instead_of_hardlinks.contains(file_name_with_projection_prefix)
&& it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME
&& it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME)
{
hardlinked_files->hardlinks_from_source_part.insert(file_name_with_projection_prefix);
}
}
}
}

View File

@ -1322,9 +1322,11 @@ private:
for (auto p_it = projection_data_part_storage_src->iterate(); p_it->isValid(); p_it->next())
{
auto file_name_with_projection_prefix = fs::path(projection_data_part_storage_src->getPartDirectory()) / p_it->name();
projection_data_part_storage_dst->createHardLinkFrom(
*projection_data_part_storage_src, p_it->name(), p_it->name());
hardlinked_files.insert(p_it->name());
hardlinked_files.insert(file_name_with_projection_prefix);
}
}
}

View File

@ -1,6 +1,6 @@
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/AsynchronousMetrics.h>
#include <Common/AsynchronousMetrics.h>
#include <Storages/System/StorageSystemAsynchronousMetrics.h>

View File

@ -24,7 +24,8 @@ NamesAndTypesList StorageSystemFilesystemCache::getNamesAndTypes()
{"cache_hits", std::make_shared<DataTypeUInt64>()},
{"references", std::make_shared<DataTypeUInt64>()},
{"downloaded_size", std::make_shared<DataTypeUInt64>()},
{"persistent", std::make_shared<DataTypeNumber<UInt8>>()}
{"persistent", std::make_shared<DataTypeNumber<UInt8>>()},
{"kind", std::make_shared<DataTypeString>()},
};
}
@ -45,8 +46,11 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex
for (const auto & file_segment : file_segments)
{
res_columns[0]->insert(cache_base_path);
/// Do not use `file_segment->getPathInLocalCache` here because it will lead to nullptr dereference
/// (because file_segments in getSnapshot doesn't have `cache` field set)
res_columns[1]->insert(
cache->getPathInLocalCache(file_segment->key(), file_segment->offset(), file_segment->isPersistent()));
cache->getPathInLocalCache(file_segment->key(), file_segment->offset(), file_segment->getKind()));
const auto & range = file_segment->range();
res_columns[2]->insert(range.left);
@ -57,6 +61,7 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex
res_columns[7]->insert(file_segment->getRefCount());
res_columns[8]->insert(file_segment->getDownloadedSize());
res_columns[9]->insert(file_segment->isPersistent());
res_columns[10]->insert(toString(file_segment->getKind()));
}
}
}

16
tests/ci/.mypy.ini Normal file
View File

@ -0,0 +1,16 @@
[mypy]
warn_no_return = False
warn_unused_configs = True
disallow_subclassing_any = True
disallow_untyped_calls = False
disallow_untyped_defs = False
disallow_incomplete_defs = True
check_untyped_defs = True
disallow_untyped_decorators = True
no_implicit_optional = True
warn_redundant_casts = True
warn_unused_ignores = True
warn_return_any = True
no_implicit_reexport = True
strict_equality = True
strict_concatenate = True

View File

@ -121,7 +121,7 @@ def check_for_success_run(
s3_prefix: str,
build_name: str,
build_config: BuildConfig,
):
) -> None:
logged_prefix = os.path.join(S3_BUILDS_BUCKET, s3_prefix)
logging.info("Checking for artifacts in %s", logged_prefix)
try:
@ -174,7 +174,7 @@ def create_json_artifact(
build_config: BuildConfig,
elapsed: int,
success: bool,
):
) -> None:
subprocess.check_call(
f"echo 'BUILD_URLS=build_urls_{build_name}' >> $GITHUB_ENV", shell=True
)
@ -218,7 +218,7 @@ def upload_master_static_binaries(
build_config: BuildConfig,
s3_helper: S3Helper,
build_output_path: str,
):
) -> None:
"""Upload binary artifacts to a static S3 links"""
static_binary_name = build_config.get("static_binary_name", False)
if pr_info.number != 0:

View File

@ -5,7 +5,7 @@ import logging
import os
import sys
import time
from typing import List, Optional
from typing import Any, List, Optional
import requests # type: ignore
@ -18,7 +18,7 @@ def get_with_retries(
url: str,
retries: int = DOWNLOAD_RETRIES_COUNT,
sleep: int = 3,
**kwargs,
**kwargs: Any,
) -> requests.Response:
logging.info(
"Getting URL with %i tries and sleep %i in between: %s", retries, sleep, url
@ -41,18 +41,18 @@ def get_with_retries(
return response
def get_build_name_for_check(check_name) -> str:
return CI_CONFIG["tests_config"][check_name]["required_build"]
def get_build_name_for_check(check_name: str) -> str:
return CI_CONFIG["tests_config"][check_name]["required_build"] # type: ignore
def read_build_urls(build_name, reports_path) -> List[str]:
def read_build_urls(build_name: str, reports_path: str) -> List[str]:
for root, _, files in os.walk(reports_path):
for f in files:
if build_name in f:
logging.info("Found build report json %s", f)
with open(os.path.join(root, f), "r", encoding="utf-8") as file_handler:
build_report = json.load(file_handler)
return build_report["build_urls"]
return build_report["build_urls"] # type: ignore
return []

View File

@ -19,7 +19,7 @@ from env_helper import (
from report import create_build_html_report
from s3_helper import S3Helper
from get_robot_token import get_best_robot_token
from pr_info import PRInfo
from pr_info import NeedsDataType, PRInfo
from commit_status_helper import (
get_commit,
update_mergeable_check,
@ -28,7 +28,7 @@ from ci_config import CI_CONFIG
from rerun_helper import RerunHelper
NEEDS_DATA_PATH = os.getenv("NEEDS_DATA_PATH")
NEEDS_DATA_PATH = os.getenv("NEEDS_DATA_PATH", "")
class BuildResult:
@ -98,7 +98,7 @@ def get_failed_report(
def process_report(
build_report,
build_report: dict,
) -> Tuple[List[BuildResult], List[List[str]], List[str]]:
build_config = build_report["build_config"]
build_result = BuildResult(
@ -144,16 +144,14 @@ def main():
os.makedirs(temp_path)
build_check_name = sys.argv[1]
needs_data = None
needs_data = {} # type: NeedsDataType
required_builds = 0
if os.path.exists(NEEDS_DATA_PATH):
with open(NEEDS_DATA_PATH, "rb") as file_handler:
needs_data = json.load(file_handler)
required_builds = len(needs_data)
if needs_data is not None and all(
i["result"] == "skipped" for i in needs_data.values()
):
if needs_data and all(i["result"] == "skipped" for i in needs_data.values()):
logging.info("All builds are skipped, exiting")
sys.exit(0)
@ -218,19 +216,21 @@ def main():
build_logs = []
for build_report in build_reports:
build_result, build_artifacts_url, build_logs_url = process_report(build_report)
logging.info(
"Got %s artifact groups for build report report", len(build_result)
_build_results, build_artifacts_url, build_logs_url = process_report(
build_report
)
build_results.extend(build_result)
logging.info(
"Got %s artifact groups for build report report", len(_build_results)
)
build_results.extend(_build_results)
build_artifacts.extend(build_artifacts_url)
build_logs.extend(build_logs_url)
for failed_job in missing_build_names:
build_result, build_artifacts_url, build_logs_url = get_failed_report(
_build_results, build_artifacts_url, build_logs_url = get_failed_report(
failed_job
)
build_results.extend(build_result)
build_results.extend(_build_results)
build_artifacts.extend(build_artifacts_url)
build_logs.extend(build_logs_url)

View File

@ -106,7 +106,7 @@ def _exec_get_with_retry(url: str, token: str) -> dict:
try:
response = requests.get(url, headers=headers)
response.raise_for_status()
return response.json()
return response.json() # type: ignore
except Exception as ex:
print("Got exception executing request", ex)
time.sleep(i + 1)
@ -130,8 +130,7 @@ WorkflowDescription = namedtuple(
def get_workflows_description_for_pull_request(
pull_request_event,
token,
pull_request_event: dict, token: str
) -> List[WorkflowDescription]:
head_repo = pull_request_event["head"]["repo"]["full_name"]
head_branch = pull_request_event["head"]["ref"]
@ -193,7 +192,7 @@ def get_workflows_description_for_pull_request(
def get_workflow_description_fallback(
pull_request_event, token
pull_request_event: dict, token: str
) -> List[WorkflowDescription]:
head_repo = pull_request_event["head"]["repo"]["full_name"]
head_branch = pull_request_event["head"]["ref"]
@ -241,7 +240,7 @@ def get_workflow_description_fallback(
WorkflowDescription(
url=wf["url"],
run_id=wf["id"],
name=workflow["name"],
name=wf["name"],
head_sha=wf["head_sha"],
status=wf["status"],
rerun_url=wf["rerun_url"],
@ -254,7 +253,7 @@ def get_workflow_description_fallback(
return workflow_descriptions
def get_workflow_description(workflow_url, token) -> WorkflowDescription:
def get_workflow_description(workflow_url: str, token: str) -> WorkflowDescription:
workflow = _exec_get_with_retry(workflow_url, token)
return WorkflowDescription(
url=workflow["url"],
@ -331,7 +330,7 @@ def main(event):
workflow_descriptions
or get_workflow_description_fallback(pull_request, token)
)
workflow_descriptions.sort(key=lambda x: x.run_id)
workflow_descriptions.sort(key=lambda x: x.run_id) # type: ignore
most_recent_workflow = workflow_descriptions[-1]
if (
most_recent_workflow.status == "completed"
@ -376,7 +375,7 @@ def main(event):
print("Not found any workflows")
return
workflow_descriptions.sort(key=lambda x: x.run_id)
workflow_descriptions.sort(key=lambda x: x.run_id) # type: ignore
most_recent_workflow = workflow_descriptions[-1]
print("Latest workflow", most_recent_workflow)
if (

View File

@ -79,7 +79,7 @@ Merge it only if you intend to backport changes to the target branch, otherwise
self.backport_pr = None # type: Optional[PullRequest]
self._backported = None # type: Optional[bool]
self.git_prefix = ( # All commits to cherrypick are done as robot-clickhouse
"git -c user.email=robot-clickhouse@clickhouse.com "
"git -c user.email=robot-clickhouse@users.noreply.github.com "
"-c user.name=robot-clickhouse -c commit.gpgsign=false"
)
self.pre_check()
@ -92,7 +92,8 @@ Merge it only if you intend to backport changes to the target branch, otherwise
if branch_updated:
self._backported = True
def pop_prs(self, prs: PullRequests):
def pop_prs(self, prs: PullRequests) -> None:
"""the method processes all prs and pops the ReleaseBranch related prs"""
to_pop = [] # type: List[int]
for i, pr in enumerate(prs):
if self.name not in pr.head.ref:
@ -105,14 +106,14 @@ Merge it only if you intend to backport changes to the target branch, otherwise
to_pop.append(i)
else:
logging.error(
"PR #%s doesn't head ref starting with known suffix",
"head ref of PR #%s isn't starting with known suffix",
pr.number,
)
for i in reversed(to_pop):
# Going from the tail to keep the order and pop greater index first
prs.pop(i)
def process(self, dry_run: bool):
def process(self, dry_run: bool) -> None:
if self.backported:
return
if not self.cherrypick_pr:
@ -209,6 +210,7 @@ Merge it only if you intend to backport changes to the target branch, otherwise
self._assign_new_pr(self.cherrypick_pr)
def create_backport(self):
assert self.cherrypick_pr is not None
# Checkout the backport branch from the remote and make all changes to
# apply like they are only one cherry-pick commit on top of release
git_runner(f"{self.git_prefix} checkout -f {self.backport_branch}")
@ -239,7 +241,7 @@ Merge it only if you intend to backport changes to the target branch, otherwise
self.backport_pr.add_to_labels(Labels.BACKPORT)
self._assign_new_pr(self.backport_pr)
def _assign_new_pr(self, new_pr: PullRequest):
def _assign_new_pr(self, new_pr: PullRequest) -> None:
"""Assign `new_pr` to author, merger and assignees of an original PR"""
# It looks there some race when multiple .add_to_assignees are executed,
# so we'll add all at once
@ -340,7 +342,7 @@ class Backport:
)
self.error = e
def process_pr(self, pr: PullRequest):
def process_pr(self, pr: PullRequest) -> None:
pr_labels = [label.name for label in pr.labels]
if Labels.MUST_BACKPORT in pr_labels:
branches = [
@ -403,7 +405,7 @@ class Backport:
# And check it after the running
self.mark_pr_backported(pr)
def mark_pr_backported(self, pr: PullRequest):
def mark_pr_backported(self, pr: PullRequest) -> None:
if self.dry_run:
logging.info("DRY RUN: would mark PR #%s as done", pr.number)
return
@ -488,7 +490,8 @@ def main():
gh = GitHub(token, per_page=100)
bp = Backport(gh, args.repo, args.dry_run)
bp.gh.cache_path = str(f"{TEMP_PATH}/gh_cache")
# https://github.com/python/mypy/issues/3004
bp.gh.cache_path = f"{TEMP_PATH}/gh_cache" # type: ignore
bp.receive_release_prs()
bp.receive_prs_for_backport()
bp.process_backports()

View File

@ -12,11 +12,12 @@ import json
import time
from collections import namedtuple
from datetime import datetime
from typing import Dict, List, Tuple
import jwt
import requests
import boto3
from botocore.exceptions import ClientError
import requests # type: ignore
import boto3 # type: ignore
from botocore.exceptions import ClientError # type: ignore
UNIVERSAL_LABEL = "universal"
RUNNER_TYPE_LABELS = [
@ -29,8 +30,13 @@ RUNNER_TYPE_LABELS = [
"style-checker-aarch64",
]
RunnerDescription = namedtuple(
"RunnerDescription", ["id", "name", "tags", "offline", "busy"]
)
RunnerDescriptions = List[RunnerDescription]
def get_dead_runners_in_ec2(runners):
def get_dead_runners_in_ec2(runners: RunnerDescriptions) -> RunnerDescriptions:
ids = {
runner.name: runner
for runner in runners
@ -92,7 +98,7 @@ def get_dead_runners_in_ec2(runners):
return result_to_delete
def get_lost_ec2_instances(runners):
def get_lost_ec2_instances(runners: RunnerDescriptions) -> List[dict]:
client = boto3.client("ec2")
reservations = client.describe_instances(
Filters=[{"Name": "tag-key", "Values": ["github:runner-type"]}]
@ -130,7 +136,7 @@ def get_lost_ec2_instances(runners):
return lost_instances
def get_key_and_app_from_aws():
def get_key_and_app_from_aws() -> Tuple[str, int]:
secret_name = "clickhouse_github_secret_key"
session = boto3.session.Session()
client = session.client(
@ -146,7 +152,7 @@ def handler(event, context):
main(private_key, app_id, True, True)
def get_installation_id(jwt_token):
def get_installation_id(jwt_token: str) -> int:
headers = {
"Authorization": f"Bearer {jwt_token}",
"Accept": "application/vnd.github.v3+json",
@ -157,10 +163,12 @@ def get_installation_id(jwt_token):
for installation in data:
if installation["account"]["login"] == "ClickHouse":
installation_id = installation["id"]
return installation_id
break
return installation_id # type: ignore
def get_access_token(jwt_token, installation_id):
def get_access_token(jwt_token: str, installation_id: int) -> str:
headers = {
"Authorization": f"Bearer {jwt_token}",
"Accept": "application/vnd.github.v3+json",
@ -171,15 +179,10 @@ def get_access_token(jwt_token, installation_id):
)
response.raise_for_status()
data = response.json()
return data["token"]
return data["token"] # type: ignore
RunnerDescription = namedtuple(
"RunnerDescription", ["id", "name", "tags", "offline", "busy"]
)
def list_runners(access_token):
def list_runners(access_token: str) -> RunnerDescriptions:
headers = {
"Authorization": f"token {access_token}",
"Accept": "application/vnd.github.v3+json",
@ -225,8 +228,10 @@ def list_runners(access_token):
return result
def group_runners_by_tag(listed_runners):
result = {}
def group_runners_by_tag(
listed_runners: RunnerDescriptions,
) -> Dict[str, RunnerDescriptions]:
result = {} # type: Dict[str, RunnerDescriptions]
def add_to_result(tag, runner):
if tag not in result:
@ -248,7 +253,9 @@ def group_runners_by_tag(listed_runners):
return result
def push_metrics_to_cloudwatch(listed_runners, namespace):
def push_metrics_to_cloudwatch(
listed_runners: RunnerDescriptions, namespace: str
) -> None:
client = boto3.client("cloudwatch")
metrics_data = []
busy_runners = sum(
@ -278,7 +285,7 @@ def push_metrics_to_cloudwatch(listed_runners, namespace):
}
)
if total_active_runners == 0:
busy_ratio = 100
busy_ratio = 100.0
else:
busy_ratio = busy_runners / total_active_runners * 100
@ -293,7 +300,7 @@ def push_metrics_to_cloudwatch(listed_runners, namespace):
client.put_metric_data(Namespace=namespace, MetricData=metrics_data)
def delete_runner(access_token, runner):
def delete_runner(access_token: str, runner: RunnerDescription) -> bool:
headers = {
"Authorization": f"token {access_token}",
"Accept": "application/vnd.github.v3+json",
@ -305,10 +312,15 @@ def delete_runner(access_token, runner):
)
response.raise_for_status()
print(f"Response code deleting {runner.name} is {response.status_code}")
return response.status_code == 204
return bool(response.status_code == 204)
def main(github_secret_key, github_app_id, push_to_cloudwatch, delete_offline_runners):
def main(
github_secret_key: str,
github_app_id: int,
push_to_cloudwatch: bool,
delete_offline_runners: bool,
) -> None:
payload = {
"iat": int(time.time()) - 60,
"exp": int(time.time()) + (10 * 60),

View File

@ -7,14 +7,21 @@ import logging
from github import Github
from env_helper import IMAGES_PATH, REPO_COPY, S3_TEST_REPORTS_BUCKET, S3_DOWNLOAD
from stopwatch import Stopwatch
from upload_result_helper import upload_results
from s3_helper import S3Helper
from get_robot_token import get_best_robot_token
from env_helper import (
IMAGES_PATH,
REPO_COPY,
S3_DOWNLOAD,
S3_TEST_REPORTS_BUCKET,
TEMP_PATH,
)
from commit_status_helper import post_commit_status
from docker_pull_helper import get_image_with_version
from get_robot_token import get_best_robot_token
from pr_info import PRInfo
from s3_helper import S3Helper
from stopwatch import Stopwatch
from tee_popen import TeePopen
from upload_result_helper import upload_results
NAME = "Woboq Build"
@ -33,17 +40,16 @@ if __name__ == "__main__":
stopwatch = Stopwatch()
temp_path = os.getenv("TEMP_PATH", os.path.abspath("."))
gh = Github(get_best_robot_token(), per_page=100)
pr_info = PRInfo()
if not os.path.exists(temp_path):
os.makedirs(temp_path)
if not os.path.exists(TEMP_PATH):
os.makedirs(TEMP_PATH)
docker_image = get_image_with_version(IMAGES_PATH, "clickhouse/codebrowser")
s3_helper = S3Helper()
result_path = os.path.join(temp_path, "result_path")
result_path = os.path.join(TEMP_PATH, "result_path")
if not os.path.exists(result_path):
os.makedirs(result_path)
@ -51,7 +57,7 @@ if __name__ == "__main__":
logging.info("Going to run codebrowser: %s", run_command)
run_log_path = os.path.join(temp_path, "runlog.log")
run_log_path = os.path.join(TEMP_PATH, "runlog.log")
with TeePopen(run_command, run_log_path) as process:
retcode = process.wait()
@ -60,7 +66,7 @@ if __name__ == "__main__":
else:
logging.info("Run failed")
subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True)
subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {TEMP_PATH}", shell=True)
report_path = os.path.join(result_path, "html_report")
logging.info("Report path %s", report_path)
@ -76,12 +82,8 @@ if __name__ == "__main__":
test_results = [(index_html, "Look at the report")]
report_url = upload_results(
s3_helper, 0, os.getenv("GITHUB_SHA"), test_results, [], NAME
)
report_url = upload_results(s3_helper, 0, pr_info.sha, test_results, [], NAME)
print(f"::notice ::Report url: {report_url}")
post_commit_status(
gh, os.getenv("GITHUB_SHA"), NAME, "Report built", "success", report_url
)
post_commit_status(gh, pr_info.sha, NAME, "Report built", "success", report_url)

View File

@ -17,7 +17,7 @@ RETRY = 5
CommitStatuses = List[CommitStatus]
def override_status(status: str, check_name: str, invert=False) -> str:
def override_status(status: str, check_name: str, invert: bool = False) -> str:
if CI_CONFIG["tests_config"].get(check_name, {}).get("force_tests", False):
return "success"
@ -45,7 +45,7 @@ def get_commit(gh: Github, commit_sha: str, retry_count: int = RETRY) -> Commit:
def post_commit_status(
gh: Github, sha: str, check_name: str, description: str, state: str, report_url: str
):
) -> None:
for i in range(RETRY):
try:
commit = get_commit(gh, sha, 1)
@ -64,7 +64,7 @@ def post_commit_status(
def post_commit_status_to_file(
file_path: str, description: str, state: str, report_url: str
):
) -> None:
if os.path.exists(file_path):
raise Exception(f'File "{file_path}" already exists!')
with open(file_path, "w", encoding="utf-8") as f:
@ -88,21 +88,21 @@ def get_commit_filtered_statuses(commit: Commit) -> CommitStatuses:
return list(filtered.values())
def remove_labels(gh: Github, pr_info: PRInfo, labels_names: List[str]):
def remove_labels(gh: Github, pr_info: PRInfo, labels_names: List[str]) -> None:
repo = gh.get_repo(GITHUB_REPOSITORY)
pull_request = repo.get_pull(pr_info.number)
for label in labels_names:
pull_request.remove_from_labels(label)
def post_labels(gh: Github, pr_info: PRInfo, labels_names: List[str]):
def post_labels(gh: Github, pr_info: PRInfo, labels_names: List[str]) -> None:
repo = gh.get_repo(GITHUB_REPOSITORY)
pull_request = repo.get_pull(pr_info.number)
for label in labels_names:
pull_request.add_to_labels(label)
def fail_mergeable_check(commit: Commit, description: str):
def fail_mergeable_check(commit: Commit, description: str) -> None:
commit.create_status(
context="Mergeable Check",
description=description,
@ -111,7 +111,7 @@ def fail_mergeable_check(commit: Commit, description: str):
)
def reset_mergeable_check(commit: Commit, description: str = ""):
def reset_mergeable_check(commit: Commit, description: str = "") -> None:
commit.create_status(
context="Mergeable Check",
description=description,
@ -120,7 +120,7 @@ def reset_mergeable_check(commit: Commit, description: str = ""):
)
def update_mergeable_check(gh: Github, pr_info: PRInfo, check_name: str):
def update_mergeable_check(gh: Github, pr_info: PRInfo, check_name: str) -> None:
if SKIP_MERGEABLE_CHECK_LABEL in pr_info.labels:
return

View File

@ -8,7 +8,7 @@ import shutil
import subprocess
import time
import sys
from typing import Dict, List, Optional, Set, Tuple, Union
from typing import Any, Dict, List, Optional, Set, Tuple, Union
from github import Github
@ -52,7 +52,7 @@ class DockerImage:
and self.only_amd64 == other.only_amd64
)
def __lt__(self, other) -> bool:
def __lt__(self, other: Any) -> bool:
if not isinstance(other, DockerImage):
return False
if self.parent and not other.parent:
@ -270,7 +270,7 @@ def build_and_push_one_image(
def process_single_image(
image: DockerImage,
versions: List[str],
additional_cache,
additional_cache: str,
push: bool,
child: bool,
) -> List[Tuple[str, str, str]]:

View File

@ -70,7 +70,7 @@ def parse_args() -> argparse.Namespace:
def load_images(path: str, suffix: str) -> Images:
with open(os.path.join(path, CHANGED_IMAGES.format(suffix)), "rb") as images:
return json.load(images)
return json.load(images) # type: ignore
def strip_suffix(suffix: str, images: Images) -> Images:

View File

@ -6,11 +6,11 @@ import time
import subprocess
import logging
from typing import Optional
from typing import List, Optional
class DockerImage:
def __init__(self, name, version: Optional[str] = None):
def __init__(self, name: str, version: Optional[str] = None):
self.name = name
if version is None:
self.version = "latest"
@ -22,8 +22,11 @@ class DockerImage:
def get_images_with_versions(
reports_path, required_image, pull=True, version: Optional[str] = None
):
reports_path: str,
required_images: List[str],
pull: bool = True,
version: Optional[str] = None,
) -> List[DockerImage]:
images_path = None
for root, _, files in os.walk(reports_path):
for f in files:
@ -45,12 +48,13 @@ def get_images_with_versions(
images = {}
docker_images = []
for image_name in required_image:
for image_name in required_images:
docker_image = DockerImage(image_name, version)
if image_name in images:
docker_image.version = images[image_name]
docker_images.append(docker_image)
latest_error = Exception("predefined to avoid access before created")
if pull:
for docker_image in docker_images:
for i in range(10):
@ -75,6 +79,8 @@ def get_images_with_versions(
return docker_images
def get_image_with_version(reports_path, image, pull=True, version=None):
def get_image_with_version(
reports_path: str, image: str, pull: bool = True, version: Optional[str] = None
) -> DockerImage:
logging.info("Looking for images file in %s", reports_path)
return get_images_with_versions(reports_path, [image], pull, version=version)[0]

View File

@ -43,55 +43,55 @@ class TestDockerImageCheck(unittest.TestCase):
"docker/test/stateless",
"clickhouse/stateless-test",
False,
"clickhouse/test-base",
"clickhouse/test-base", # type: ignore
),
di.DockerImage(
"docker/test/integration/base",
"clickhouse/integration-test",
False,
"clickhouse/test-base",
"clickhouse/test-base", # type: ignore
),
di.DockerImage(
"docker/test/fuzzer",
"clickhouse/fuzzer",
False,
"clickhouse/test-base",
"clickhouse/test-base", # type: ignore
),
di.DockerImage(
"docker/test/keeper-jepsen",
"clickhouse/keeper-jepsen-test",
False,
"clickhouse/test-base",
"clickhouse/test-base", # type: ignore
),
di.DockerImage(
"docker/docs/check",
"clickhouse/docs-check",
False,
"clickhouse/docs-builder",
"clickhouse/docs-builder", # type: ignore
),
di.DockerImage(
"docker/docs/release",
"clickhouse/docs-release",
False,
"clickhouse/docs-builder",
"clickhouse/docs-builder", # type: ignore
),
di.DockerImage(
"docker/test/stateful",
"clickhouse/stateful-test",
False,
"clickhouse/stateless-test",
"clickhouse/stateless-test", # type: ignore
),
di.DockerImage(
"docker/test/unit",
"clickhouse/unit-test",
False,
"clickhouse/stateless-test",
"clickhouse/stateless-test", # type: ignore
),
di.DockerImage(
"docker/test/stress",
"clickhouse/stress-test",
False,
"clickhouse/stateful-test",
"clickhouse/stateful-test", # type: ignore
),
]
)
@ -277,7 +277,7 @@ class TestDockerServer(unittest.TestCase):
ds.gen_tags(version, "auto")
@patch("docker_server.get_tagged_versions")
def test_auto_release_type(self, mock_tagged_versions: MagicMock):
def test_auto_release_type(self, mock_tagged_versions: MagicMock) -> None:
mock_tagged_versions.return_value = [
get_version_from_string("1.1.1.1"),
get_version_from_string("1.2.1.1"),

View File

@ -6,6 +6,7 @@ import os
import csv
import sys
import atexit
from typing import List, Tuple
from github import Github
@ -50,8 +51,10 @@ def get_fasttest_cmd(
)
def process_results(result_folder):
test_results = []
def process_results(
result_folder: str,
) -> Tuple[str, str, List[Tuple[str, str]], List[str]]:
test_results = [] # type: List[Tuple[str, str]]
additional_files = []
# Just upload all files from result_folder.
# If task provides processed results, then it's responsible for content of
@ -78,7 +81,7 @@ def process_results(result_folder):
results_path = os.path.join(result_folder, "test_results.tsv")
if os.path.exists(results_path):
with open(results_path, "r", encoding="utf-8") as results_file:
test_results = list(csv.reader(results_file, delimiter="\t"))
test_results = list(csv.reader(results_file, delimiter="\t")) # type: ignore
if len(test_results) == 0:
return "error", "Empty test_results.tsv", test_results, additional_files
@ -172,7 +175,7 @@ if __name__ == "__main__":
"test_log.txt" in test_output_files or "test_result.txt" in test_output_files
)
test_result_exists = "test_results.tsv" in test_output_files
test_results = []
test_results = [] # type: List[Tuple[str, str]]
if "submodule_log.txt" not in test_output_files:
description = "Cannot clone repository"
state = "failure"

View File

@ -5,27 +5,11 @@ from github import Github
from env_helper import GITHUB_RUN_URL
from pr_info import PRInfo
from get_robot_token import get_best_robot_token
from commit_status_helper import get_commit
from commit_status_helper import get_commit, get_commit_filtered_statuses
NAME = "Run Check"
def filter_statuses(statuses):
"""
Squash statuses to latest state
1. context="first", state="success", update_time=1
2. context="second", state="success", update_time=2
3. context="first", stat="failure", update_time=3
=========>
1. context="second", state="success"
2. context="first", stat="failure"
"""
filt = {}
for status in sorted(statuses, key=lambda x: x.updated_at):
filt[status.context] = status
return filt
if __name__ == "__main__":
logging.basicConfig(level=logging.INFO)
@ -34,8 +18,13 @@ if __name__ == "__main__":
commit = get_commit(gh, pr_info.sha)
url = GITHUB_RUN_URL
statuses = filter_statuses(list(commit.get_statuses()))
if NAME in statuses and statuses[NAME].state == "pending":
statuses = get_commit_filtered_statuses(commit)
pending_status = any( # find NAME status in pending state
True
for status in statuses
if status.context == NAME and status.state == "pending"
)
if pending_status:
commit.create_status(
context=NAME,
description="All checks finished",

View File

@ -7,6 +7,7 @@ import os
import subprocess
import sys
import atexit
from typing import List, Tuple
from github import Github
@ -122,8 +123,11 @@ def get_tests_to_run(pr_info):
return list(result)
def process_results(result_folder, server_log_path):
test_results = []
def process_results(
result_folder: str,
server_log_path: str,
) -> Tuple[str, str, List[Tuple[str, str]], List[str]]:
test_results = [] # type: List[Tuple[str, str]]
additional_files = []
# Just upload all files from result_folder.
# If task provides processed results, then it's responsible for content of result_folder.
@ -166,7 +170,7 @@ def process_results(result_folder, server_log_path):
return "error", "Not found test_results.tsv", test_results, additional_files
with open(results_path, "r", encoding="utf-8") as results_file:
test_results = list(csv.reader(results_file, delimiter="\t"))
test_results = list(csv.reader(results_file, delimiter="\t")) # type: ignore
if len(test_results) == 0:
return "error", "Empty test_results.tsv", test_results, additional_files
@ -232,8 +236,8 @@ if __name__ == "__main__":
sys.exit(0)
if "RUN_BY_HASH_NUM" in os.environ:
run_by_hash_num = int(os.getenv("RUN_BY_HASH_NUM"))
run_by_hash_total = int(os.getenv("RUN_BY_HASH_TOTAL"))
run_by_hash_num = int(os.getenv("RUN_BY_HASH_NUM", "0"))
run_by_hash_total = int(os.getenv("RUN_BY_HASH_TOTAL", "0"))
check_name_with_group = (
check_name + f" [{run_by_hash_num + 1}/{run_by_hash_total}]"
)

Some files were not shown because too many files have changed in this diff Show More