mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Merge branch 'master' into parallel-log-appending
This commit is contained in:
commit
7cd76df918
@ -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))
|
||||
|
||||
|
@ -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
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
|
@ -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.
|
||||
|
@ -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})
|
||||
|
@ -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();
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include "MetricsTransmitter.h"
|
||||
|
||||
#include <Interpreters/AsynchronousMetrics.h>
|
||||
#include <Common/AsynchronousMetrics.h>
|
||||
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/Exception.h>
|
||||
|
@ -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,
|
||||
|
@ -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,
|
||||
|
@ -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;
|
||||
|
@ -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,
|
||||
|
@ -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." };
|
||||
}
|
||||
|
||||
}
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
@ -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.") \
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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()
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
|
127
src/Coordination/KeeperAsynchronousMetrics.cpp
Normal file
127
src/Coordination/KeeperAsynchronousMetrics.cpp
Normal 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
|
||||
}
|
||||
|
||||
}
|
25
src/Coordination/KeeperAsynchronousMetrics.h
Normal file
25
src/Coordination/KeeperAsynchronousMetrics.h
Normal 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;
|
||||
};
|
||||
|
||||
|
||||
}
|
@ -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);
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include "TinyContext.h"
|
||||
#include <Coordination/TinyContext.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <Coordination/KeeperDispatcher.h>
|
@ -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>;
|
||||
|
||||
}
|
@ -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);
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
73
src/Disks/IO/FileCachePlaceholder.cpp
Normal file
73
src/Disks/IO/FileCachePlaceholder.cpp
Normal 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));
|
||||
}
|
||||
|
||||
|
||||
}
|
61
src/Disks/IO/FileCachePlaceholder.h
Normal file
61
src/Disks/IO/FileCachePlaceholder.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
@ -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;
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
|
20
src/Disks/TemporaryFileInPath.cpp
Normal file
20
src/Disks/TemporaryFileInPath.cpp
Normal 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;
|
||||
|
||||
}
|
21
src/Disks/TemporaryFileInPath.h
Normal file
21
src/Disks/TemporaryFileInPath.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
@ -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
|
||||
{
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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"; }
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
|
||||
|
@ -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;
|
||||
};
|
||||
|
@ -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
|
||||
|
@ -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>
|
||||
|
@ -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);
|
||||
|
@ -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,
|
||||
|
@ -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(
|
||||
|
@ -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();
|
||||
|
@ -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;
|
||||
|
@ -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();
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
395
src/Interpreters/ServerAsynchronousMetrics.cpp
Normal file
395
src/Interpreters/ServerAsynchronousMetrics.cpp
Normal 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." };
|
||||
}
|
||||
|
||||
}
|
37
src/Interpreters/ServerAsynchronousMetrics.h
Normal file
37
src/Interpreters/ServerAsynchronousMetrics.h
Normal 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);
|
||||
};
|
||||
|
||||
}
|
@ -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()
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
|
24
src/Server/HTTP/HTTPContext.h
Normal file
24
src/Server/HTTP/HTTPContext.h
Normal 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>;
|
||||
|
||||
}
|
@ -6,7 +6,7 @@
|
||||
namespace DB
|
||||
{
|
||||
HTTPServer::HTTPServer(
|
||||
ContextPtr context,
|
||||
HTTPContextPtr context,
|
||||
HTTPRequestHandlerFactoryPtr factory_,
|
||||
Poco::ThreadPool & thread_pool,
|
||||
Poco::Net::ServerSocket & socket_,
|
||||
|
@ -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,
|
||||
|
@ -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));
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
@ -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)
|
||||
|
@ -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,
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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)
|
||||
|
38
src/Server/HTTPRequestHandlerFactoryMain.cpp
Normal file
38
src/Server/HTTPRequestHandlerFactoryMain.cpp
Normal 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;
|
||||
}
|
||||
|
||||
}
|
27
src/Server/HTTPRequestHandlerFactoryMain.h
Normal file
27
src/Server/HTTPRequestHandlerFactoryMain.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <string>
|
||||
|
||||
#include <Interpreters/AsynchronousMetrics.h>
|
||||
#include <Common/AsynchronousMetrics.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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.
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
|
@ -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};
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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>
|
||||
|
||||
|
||||
|
@ -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
16
tests/ci/.mypy.ini
Normal 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
|
@ -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:
|
||||
|
@ -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 []
|
||||
|
||||
|
||||
|
@ -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)
|
||||
|
||||
|
@ -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 (
|
||||
|
@ -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()
|
||||
|
@ -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),
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
|
||||
|
@ -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]]:
|
||||
|
@ -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:
|
||||
|
@ -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]
|
||||
|
@ -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"),
|
||||
|
@ -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"
|
||||
|
@ -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",
|
||||
|
@ -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
Loading…
Reference in New Issue
Block a user