mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 18:12:02 +00:00
Merge branch 'master' into new-json-formats
This commit is contained in:
commit
faa62f96ae
@ -38,6 +38,7 @@ FORMAT_SCHEMA_PATH="$(clickhouse extract-from-config --config-file "$CLICKHOUSE_
|
||||
|
||||
# There could be many disks declared in config
|
||||
readarray -t DISKS_PATHS < <(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key='storage_configuration.disks.*.path' || true)
|
||||
readarray -t DISKS_METADATA_PATHS < <(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key='storage_configuration.disks.*.metadata_path' || true)
|
||||
|
||||
CLICKHOUSE_USER="${CLICKHOUSE_USER:-default}"
|
||||
CLICKHOUSE_PASSWORD="${CLICKHOUSE_PASSWORD:-}"
|
||||
@ -50,7 +51,8 @@ for dir in "$DATA_DIR" \
|
||||
"$TMP_DIR" \
|
||||
"$USER_PATH" \
|
||||
"$FORMAT_SCHEMA_PATH" \
|
||||
"${DISKS_PATHS[@]}"
|
||||
"${DISKS_PATHS[@]}" \
|
||||
"${DISKS_METADATA_PATHS[@]}"
|
||||
do
|
||||
# check if variable not empty
|
||||
[ -z "$dir" ] && continue
|
||||
@ -106,7 +108,7 @@ if [ -n "$(ls /docker-entrypoint-initdb.d/)" ] || [ -n "$CLICKHOUSE_DB" ]; then
|
||||
# port is needed to check if clickhouse-server is ready for connections
|
||||
HTTP_PORT="$(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key=http_port)"
|
||||
HTTPS_PORT="$(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key=https_port)"
|
||||
|
||||
|
||||
if [ -n "$HTTP_PORT" ]; then
|
||||
URL="http://127.0.0.1:$HTTP_PORT/ping"
|
||||
else
|
||||
|
@ -370,6 +370,7 @@ else
|
||||
|
||||
# Avoid "Setting s3_check_objects_after_upload is neither a builtin setting..."
|
||||
rm -f /etc/clickhouse-server/users.d/enable_blobs_check.xml ||:
|
||||
rm -f /etc/clickhouse-server/users.d/marks.xml ||:
|
||||
|
||||
# Remove s3 related configs to avoid "there is no disk type `cache`"
|
||||
rm -f /etc/clickhouse-server/config.d/storage_conf.xml ||:
|
||||
|
@ -227,6 +227,8 @@ void LocalServer::cleanup()
|
||||
global_context.reset();
|
||||
}
|
||||
|
||||
/// thread status should be destructed before shared context because it relies on process list.
|
||||
|
||||
status.reset();
|
||||
|
||||
// Delete the temporary directory if needed.
|
||||
@ -366,7 +368,7 @@ int LocalServer::main(const std::vector<std::string> & /*args*/)
|
||||
try
|
||||
{
|
||||
UseSSL use_ssl;
|
||||
ThreadStatus thread_status;
|
||||
thread_status.emplace();
|
||||
|
||||
StackTrace::setShowAddresses(config().getBool("show_addresses_in_stack_traces", true));
|
||||
|
||||
|
@ -40,6 +40,7 @@
|
||||
#include <Common/getMappedArea.h>
|
||||
#include <Common/remapExecutable.h>
|
||||
#include <Common/TLDListsHolder.h>
|
||||
#include <Common/Config/AbstractConfigurationComparison.h>
|
||||
#include <Core/ServerUUID.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
@ -1269,6 +1270,9 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
CertificateReloader::instance().tryLoad(*config);
|
||||
#endif
|
||||
ProfileEvents::increment(ProfileEvents::MainConfigLoads);
|
||||
|
||||
/// Must be the last.
|
||||
latest_config = config;
|
||||
},
|
||||
/* already_loaded = */ false); /// Reload it right now (initial loading)
|
||||
|
||||
@ -1886,7 +1890,7 @@ void Server::createServers(
|
||||
port_name,
|
||||
"http://" + address.toString(),
|
||||
std::make_unique<HTTPServer>(
|
||||
context(), createHandlerFactory(*this, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params));
|
||||
context(), createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params));
|
||||
});
|
||||
|
||||
/// HTTPS
|
||||
@ -1903,7 +1907,7 @@ void Server::createServers(
|
||||
port_name,
|
||||
"https://" + address.toString(),
|
||||
std::make_unique<HTTPServer>(
|
||||
context(), createHandlerFactory(*this, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params));
|
||||
context(), 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.",
|
||||
@ -2028,7 +2032,7 @@ void Server::createServers(
|
||||
port_name,
|
||||
"Prometheus: http://" + address.toString(),
|
||||
std::make_unique<HTTPServer>(
|
||||
context(), createHandlerFactory(*this, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params));
|
||||
context(), createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params));
|
||||
});
|
||||
}
|
||||
|
||||
@ -2049,7 +2053,7 @@ void Server::createServers(
|
||||
"replica communication (interserver): http://" + address.toString(),
|
||||
std::make_unique<HTTPServer>(
|
||||
context(),
|
||||
createHandlerFactory(*this, async_metrics, "InterserverIOHTTPHandler-factory"),
|
||||
createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory"),
|
||||
server_pool,
|
||||
socket,
|
||||
http_params));
|
||||
@ -2069,7 +2073,7 @@ void Server::createServers(
|
||||
"secure replica communication (interserver): https://" + address.toString(),
|
||||
std::make_unique<HTTPServer>(
|
||||
context(),
|
||||
createHandlerFactory(*this, async_metrics, "InterserverIOHTTPSHandler-factory"),
|
||||
createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPSHandler-factory"),
|
||||
server_pool,
|
||||
socket,
|
||||
http_params));
|
||||
@ -2111,13 +2115,24 @@ void Server::updateServers(
|
||||
|
||||
std::erase_if(servers, std::bind_front(check_server, " (from one of previous reload)"));
|
||||
|
||||
Poco::Util::AbstractConfiguration & previous_config = latest_config ? *latest_config : this->config();
|
||||
|
||||
for (auto & server : servers)
|
||||
{
|
||||
if (!server.isStopping())
|
||||
{
|
||||
bool has_host = std::find(listen_hosts.begin(), listen_hosts.end(), server.getListenHost()) != listen_hosts.end();
|
||||
bool has_port = !config.getString(server.getPortName(), "").empty();
|
||||
if (!has_host || !has_port || config.getInt(server.getPortName()) != server.portNumber())
|
||||
|
||||
/// NOTE: better to compare using getPortName() over using
|
||||
/// dynamic_cast<> since HTTPServer is also used for prometheus and
|
||||
/// internal replication communications.
|
||||
bool is_http = server.getPortName() == "http_port" || server.getPortName() == "https_port";
|
||||
bool force_restart = is_http && !isSameConfiguration(previous_config, config, "http_handlers");
|
||||
if (force_restart)
|
||||
LOG_TRACE(log, "<http_handlers> had been changed, will reload {}", server.getDescription());
|
||||
|
||||
if (!has_host || !has_port || config.getInt(server.getPortName()) != server.portNumber() || force_restart)
|
||||
{
|
||||
server.stop();
|
||||
LOG_INFO(log, "Stopped listening for {}", server.getDescription());
|
||||
|
@ -67,6 +67,9 @@ protected:
|
||||
|
||||
private:
|
||||
ContextMutablePtr global_context;
|
||||
/// Updated/recent config, to compare http_handlers
|
||||
ConfigurationPtr latest_config;
|
||||
|
||||
Poco::Net::SocketAddress socketBindListen(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
Poco::Net::ServerSocket & socket,
|
||||
|
@ -248,6 +248,7 @@ BackupCoordinationReplicatedTables::getMutations(const String & table_shared_id,
|
||||
return {};
|
||||
|
||||
std::vector<MutationInfo> res;
|
||||
res.reserve(table_info.mutations.size());
|
||||
for (const auto & [mutation_id, mutation_entry] : table_info.mutations)
|
||||
res.emplace_back(MutationInfo{mutation_id, mutation_entry});
|
||||
return res;
|
||||
|
@ -176,9 +176,6 @@ protected:
|
||||
bool stderr_is_a_tty = false; /// stderr is a terminal.
|
||||
uint64_t terminal_width = 0;
|
||||
|
||||
ServerConnectionPtr connection;
|
||||
ConnectionParameters connection_parameters;
|
||||
|
||||
String format; /// Query results output format.
|
||||
bool select_into_file = false; /// If writing result INTO OUTFILE. It affects progress rendering.
|
||||
bool select_into_file_and_stdout = false; /// If writing result INTO OUTFILE AND STDOUT. It affects progress rendering.
|
||||
@ -199,6 +196,11 @@ protected:
|
||||
SharedContextHolder shared_context;
|
||||
ContextMutablePtr global_context;
|
||||
|
||||
std::optional<ThreadStatus> thread_status;
|
||||
|
||||
ServerConnectionPtr connection;
|
||||
ConnectionParameters connection_parameters;
|
||||
|
||||
/// Buffer that reads from stdin in batch mode.
|
||||
ReadBufferFromFileDescriptor std_in{STDIN_FILENO};
|
||||
/// Console output.
|
||||
|
@ -16,7 +16,6 @@ namespace ErrorCodes
|
||||
extern const int ATTEMPT_TO_READ_AFTER_EOF;
|
||||
extern const int NETWORK_ERROR;
|
||||
extern const int SOCKET_TIMEOUT;
|
||||
extern const int DNS_ERROR;
|
||||
}
|
||||
|
||||
ConnectionEstablisher::ConnectionEstablisher(
|
||||
@ -91,7 +90,6 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std::
|
||||
catch (const Exception & e)
|
||||
{
|
||||
if (e.code() != ErrorCodes::NETWORK_ERROR && e.code() != ErrorCodes::SOCKET_TIMEOUT
|
||||
&& e.code() != ErrorCodes::DNS_ERROR
|
||||
&& e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF)
|
||||
throw;
|
||||
|
||||
|
@ -31,9 +31,6 @@ LocalConnection::LocalConnection(ContextPtr context_, bool send_progress_, bool
|
||||
/// Authenticate and create a context to execute queries.
|
||||
session.authenticate("default", "", Poco::Net::SocketAddress{});
|
||||
session.makeSessionContext();
|
||||
|
||||
if (!CurrentThread::isInitialized())
|
||||
thread_status.emplace();
|
||||
}
|
||||
|
||||
LocalConnection::~LocalConnection()
|
||||
|
@ -156,7 +156,6 @@ private:
|
||||
String description = "clickhouse-local";
|
||||
|
||||
std::optional<LocalQueryState> state;
|
||||
std::optional<ThreadStatus> thread_status;
|
||||
|
||||
/// Last "server" packet.
|
||||
std::optional<UInt64> next_packet_type;
|
||||
|
33
src/Common/Base64.cpp
Normal file
33
src/Common/Base64.cpp
Normal file
@ -0,0 +1,33 @@
|
||||
#include <Common/Base64.h>
|
||||
|
||||
#include <Poco/Base64Decoder.h>
|
||||
#include <Poco/Base64Encoder.h>
|
||||
#include <Poco/MemoryStream.h>
|
||||
#include <Poco/StreamCopier.h>
|
||||
|
||||
#include <sstream>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
std::string base64Encode(const std::string & decoded, bool url_encoding)
|
||||
{
|
||||
std::ostringstream ostr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
ostr.exceptions(std::ios::failbit);
|
||||
Poco::Base64Encoder encoder(ostr, url_encoding ? Poco::BASE64_URL_ENCODING : 0);
|
||||
encoder.rdbuf()->setLineLength(0);
|
||||
encoder << decoded;
|
||||
encoder.close();
|
||||
return ostr.str();
|
||||
}
|
||||
|
||||
std::string base64Decode(const std::string & encoded, bool url_encoding)
|
||||
{
|
||||
std::string decoded;
|
||||
Poco::MemoryInputStream istr(encoded.data(), encoded.size());
|
||||
Poco::Base64Decoder decoder(istr, url_encoding ? Poco::BASE64_URL_ENCODING : 0);
|
||||
Poco::StreamCopier::copyToString(decoder, decoded);
|
||||
return decoded;
|
||||
}
|
||||
|
||||
}
|
12
src/Common/Base64.h
Normal file
12
src/Common/Base64.h
Normal file
@ -0,0 +1,12 @@
|
||||
#pragma once
|
||||
|
||||
#include <string>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
std::string base64Encode(const std::string & decoded, bool url_encoding = false);
|
||||
|
||||
std::string base64Decode(const std::string & encoded, bool url_encoding = false);
|
||||
|
||||
}
|
@ -636,6 +636,7 @@
|
||||
M(665, CANNOT_CONNECT_NATS) \
|
||||
M(666, CANNOT_USE_CACHE) \
|
||||
M(667, NOT_INITIALIZED) \
|
||||
M(668, INVALID_STATE) \
|
||||
\
|
||||
M(999, KEEPER_EXCEPTION) \
|
||||
M(1000, POCO_EXCEPTION) \
|
||||
|
@ -146,6 +146,9 @@
|
||||
M(SelectedRows, "Number of rows SELECTed from all tables.") \
|
||||
M(SelectedBytes, "Number of bytes (uncompressed; for columns as they stored in memory) SELECTed from all tables.") \
|
||||
\
|
||||
M(WaitMarksLoadMicroseconds, "Time spent loading marks") \
|
||||
M(BackgroundLoadingMarksTasks, "Number of background tasks for loading marks") \
|
||||
\
|
||||
M(Merge, "Number of launched background merges.") \
|
||||
M(MergedRows, "Rows read for background merges. This is the number of rows before merge.") \
|
||||
M(MergedUncompressedBytes, "Uncompressed bytes (for columns as they stored in memory) that was read for background merges. This is the number before merge.") \
|
||||
|
@ -605,7 +605,7 @@ void ZooKeeper::removeChildren(const std::string & path)
|
||||
}
|
||||
|
||||
|
||||
void ZooKeeper::removeChildrenRecursive(const std::string & path, const String & keep_child_node)
|
||||
void ZooKeeper::removeChildrenRecursive(const std::string & path, RemoveException keep_child)
|
||||
{
|
||||
Strings children = getChildren(path);
|
||||
while (!children.empty())
|
||||
@ -613,16 +613,23 @@ void ZooKeeper::removeChildrenRecursive(const std::string & path, const String &
|
||||
Coordination::Requests ops;
|
||||
for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i)
|
||||
{
|
||||
removeChildrenRecursive(fs::path(path) / children.back());
|
||||
if (likely(keep_child_node.empty() || keep_child_node != children.back()))
|
||||
if (keep_child.path.empty() || keep_child.path != children.back()) [[likely]]
|
||||
{
|
||||
removeChildrenRecursive(fs::path(path) / children.back());
|
||||
ops.emplace_back(makeRemoveRequest(fs::path(path) / children.back(), -1));
|
||||
}
|
||||
else if (keep_child.remove_subtree)
|
||||
{
|
||||
removeChildrenRecursive(fs::path(path) / children.back());
|
||||
}
|
||||
|
||||
children.pop_back();
|
||||
}
|
||||
multi(ops);
|
||||
}
|
||||
}
|
||||
|
||||
bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probably_flat, const String & keep_child_node)
|
||||
bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probably_flat, RemoveException keep_child)
|
||||
{
|
||||
Strings children;
|
||||
if (tryGetChildren(path, children) != Coordination::Error::ZOK)
|
||||
@ -639,16 +646,20 @@ bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probab
|
||||
{
|
||||
String child_path = fs::path(path) / children.back();
|
||||
|
||||
/// Will try to avoid recursive getChildren calls if child_path probably has no children.
|
||||
/// It may be extremely slow when path contain a lot of leaf children.
|
||||
if (!probably_flat)
|
||||
tryRemoveChildrenRecursive(child_path);
|
||||
|
||||
if (likely(keep_child_node.empty() || keep_child_node != children.back()))
|
||||
if (keep_child.path.empty() || keep_child.path != children.back()) [[likely]]
|
||||
{
|
||||
/// Will try to avoid recursive getChildren calls if child_path probably has no children.
|
||||
/// It may be extremely slow when path contain a lot of leaf children.
|
||||
if (!probably_flat)
|
||||
tryRemoveChildrenRecursive(child_path);
|
||||
|
||||
batch.push_back(child_path);
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(child_path, -1));
|
||||
}
|
||||
else if (keep_child.remove_subtree && !probably_flat)
|
||||
{
|
||||
tryRemoveChildrenRecursive(child_path);
|
||||
}
|
||||
|
||||
children.pop_back();
|
||||
}
|
||||
|
@ -58,6 +58,18 @@ struct ShuffleHost
|
||||
}
|
||||
};
|
||||
|
||||
struct RemoveException
|
||||
{
|
||||
explicit RemoveException(std::string_view path_ = "", bool remove_subtree_ = true)
|
||||
: path(path_)
|
||||
, remove_subtree(remove_subtree_)
|
||||
{}
|
||||
|
||||
std::string_view path;
|
||||
// whether we should keep the child node and its subtree or just the child node
|
||||
bool remove_subtree;
|
||||
};
|
||||
|
||||
using GetPriorityForLoadBalancing = DB::GetPriorityForLoadBalancing;
|
||||
|
||||
/// ZooKeeper session. The interface is substantially different from the usual libzookeeper API.
|
||||
@ -219,13 +231,13 @@ public:
|
||||
void tryRemoveRecursive(const std::string & path);
|
||||
|
||||
/// Similar to removeRecursive(...) and tryRemoveRecursive(...), but does not remove path itself.
|
||||
/// If keep_child_node is not empty, this method will not remove path/keep_child_node (but will remove its subtree).
|
||||
/// It can be useful to keep some child node as a flag which indicates that path is currently removing.
|
||||
void removeChildrenRecursive(const std::string & path, const String & keep_child_node = {});
|
||||
/// Node defined as RemoveException will not be deleted.
|
||||
void removeChildrenRecursive(const std::string & path, RemoveException keep_child = RemoveException{});
|
||||
/// If probably_flat is true, this method will optimistically try to remove children non-recursive
|
||||
/// and will fall back to recursive removal if it gets ZNOTEMPTY for some child.
|
||||
/// Returns true if no kind of fallback happened.
|
||||
bool tryRemoveChildrenRecursive(const std::string & path, bool probably_flat = false, const String & keep_child_node = {});
|
||||
/// Node defined as RemoveException will not be deleted.
|
||||
bool tryRemoveChildrenRecursive(const std::string & path, bool probably_flat = false, RemoveException keep_child= RemoveException{});
|
||||
|
||||
/// Remove all children nodes (non recursive).
|
||||
void removeChildren(const std::string & path);
|
||||
|
@ -442,9 +442,9 @@ void KeeperServer::shutdownRaftServer()
|
||||
|
||||
void KeeperServer::shutdown()
|
||||
{
|
||||
state_machine->shutdownStorage();
|
||||
state_manager->flushAndShutDownLogStore();
|
||||
shutdownRaftServer();
|
||||
state_machine->shutdownStorage();
|
||||
}
|
||||
|
||||
namespace
|
||||
@ -617,7 +617,9 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ
|
||||
auto & entry_buf = entry->get_buf();
|
||||
auto request_for_session = state_machine->parseRequest(entry_buf);
|
||||
request_for_session.zxid = next_zxid;
|
||||
state_machine->preprocess(request_for_session);
|
||||
if (!state_machine->preprocess(request_for_session))
|
||||
return nuraft::cb_func::ReturnCode::ReturnNull;
|
||||
|
||||
request_for_session.digest = state_machine->getNodesDigest();
|
||||
entry = nuraft::cs_new<nuraft::log_entry>(entry->get_term(), getZooKeeperLogEntry(request_for_session), entry->get_val_type());
|
||||
break;
|
||||
|
@ -191,12 +191,16 @@ KeeperStorage::RequestForSession KeeperStateMachine::parseRequest(nuraft::buffer
|
||||
return request_for_session;
|
||||
}
|
||||
|
||||
void KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & request_for_session)
|
||||
bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & request_for_session)
|
||||
{
|
||||
if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID)
|
||||
return;
|
||||
return true;
|
||||
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
|
||||
if (storage->isFinalized())
|
||||
return false;
|
||||
|
||||
try
|
||||
{
|
||||
storage->preprocessRequest(
|
||||
@ -215,6 +219,8 @@ void KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req
|
||||
|
||||
if (keeper_context->digest_enabled && request_for_session.digest)
|
||||
assertDigest(*request_for_session.digest, storage->getNodesDigest(false), *request_for_session.request, false);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data)
|
||||
|
@ -33,7 +33,7 @@ public:
|
||||
|
||||
static KeeperStorage::RequestForSession parseRequest(nuraft::buffer & data);
|
||||
|
||||
void preprocess(const KeeperStorage::RequestForSession & request_for_session);
|
||||
bool preprocess(const KeeperStorage::RequestForSession & request_for_session);
|
||||
|
||||
nuraft::ptr<nuraft::buffer> pre_commit(uint64_t log_idx, nuraft::buffer & data) override;
|
||||
|
||||
|
@ -1,11 +1,11 @@
|
||||
#include <iterator>
|
||||
#include <variant>
|
||||
#include <Coordination/KeeperStorage.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <boost/algorithm/string.hpp>
|
||||
#include <Poco/Base64Encoder.h>
|
||||
#include <Poco/SHA1Engine.h>
|
||||
|
||||
#include <Common/Base64.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperCommon.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperConstants.h>
|
||||
@ -15,8 +15,11 @@
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/LockMemoryExceptionInThread.h>
|
||||
|
||||
#include <Coordination/pathUtils.h>
|
||||
#include <Coordination/KeeperConstants.h>
|
||||
#include <Coordination/KeeperStorage.h>
|
||||
|
||||
#include <sstream>
|
||||
#include <iomanip>
|
||||
#include <mutex>
|
||||
@ -36,17 +39,6 @@ namespace ErrorCodes
|
||||
namespace
|
||||
{
|
||||
|
||||
String base64Encode(const String & decoded)
|
||||
{
|
||||
std::ostringstream ostr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
ostr.exceptions(std::ios::failbit);
|
||||
Poco::Base64Encoder encoder(ostr);
|
||||
encoder.rdbuf()->setLineLength(0);
|
||||
encoder << decoded;
|
||||
encoder.close();
|
||||
return ostr.str();
|
||||
}
|
||||
|
||||
String getSHA1(const String & userdata)
|
||||
{
|
||||
Poco::SHA1Engine engine;
|
||||
@ -516,7 +508,7 @@ void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid)
|
||||
|
||||
std::shared_ptr<KeeperStorage::Node> KeeperStorage::UncommittedState::getNode(StringRef path) const
|
||||
{
|
||||
if (auto node_it = nodes.find(std::string{path}); node_it != nodes.end())
|
||||
if (auto node_it = nodes.find(path.toView()); node_it != nodes.end())
|
||||
return node_it->second.node;
|
||||
|
||||
return tryGetNodeFromStorage(path);
|
||||
@ -524,7 +516,7 @@ std::shared_ptr<KeeperStorage::Node> KeeperStorage::UncommittedState::getNode(St
|
||||
|
||||
Coordination::ACLs KeeperStorage::UncommittedState::getACLs(StringRef path) const
|
||||
{
|
||||
if (auto node_it = nodes.find(std::string{path}); node_it != nodes.end())
|
||||
if (auto node_it = nodes.find(path.toView()); node_it != nodes.end())
|
||||
return node_it->second.acls;
|
||||
|
||||
auto node_it = storage.container.find(path);
|
||||
@ -830,7 +822,9 @@ bool KeeperStorage::checkACL(StringRef path, int32_t permission, int64_t session
|
||||
void KeeperStorage::unregisterEphemeralPath(int64_t session_id, const std::string & path)
|
||||
{
|
||||
auto ephemerals_it = ephemerals.find(session_id);
|
||||
assert(ephemerals_it != ephemerals.end());
|
||||
if (ephemerals_it == ephemerals.end())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Session {} is missing ephemeral path");
|
||||
|
||||
ephemerals_it->second.erase(path);
|
||||
if (ephemerals_it->second.empty())
|
||||
ephemerals.erase(ephemerals_it);
|
||||
@ -1825,6 +1819,11 @@ void KeeperStorage::finalize()
|
||||
session_expiry_queue.clear();
|
||||
}
|
||||
|
||||
bool KeeperStorage::isFinalized() const
|
||||
{
|
||||
return finalized;
|
||||
}
|
||||
|
||||
|
||||
class KeeperStorageRequestProcessorsFactory final : private boost::noncopyable
|
||||
{
|
||||
@ -1892,7 +1891,7 @@ UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vec
|
||||
if (!keeper_context->digest_enabled)
|
||||
return current_digest;
|
||||
|
||||
std::unordered_map<std::string, std::shared_ptr<Node>> updated_nodes;
|
||||
std::unordered_map<std::string_view, std::shared_ptr<Node>> updated_nodes;
|
||||
|
||||
for (const auto & delta : new_deltas)
|
||||
{
|
||||
|
@ -425,6 +425,8 @@ public:
|
||||
|
||||
void finalize();
|
||||
|
||||
bool isFinalized() const;
|
||||
|
||||
/// Set of methods for creating snapshots
|
||||
|
||||
/// Turn on snapshot mode, so data inside Container is not deleted, but replaced with new version.
|
||||
|
@ -602,6 +602,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
|
||||
M(Bool, skip_download_if_exceeds_query_cache, true, "Skip download from remote filesystem if exceeds query cache size", 0) \
|
||||
M(UInt64, max_query_cache_size, (128UL * 1024 * 1024 * 1024), "Max remote filesystem cache size that can be used by a single query", 0) \
|
||||
\
|
||||
M(Bool, load_marks_asynchronously, false, "Load MergeTree marks asynchronously", 0) \
|
||||
\
|
||||
M(Bool, use_structure_from_insertion_table_in_table_functions, false, "Use structure from insertion table instead of schema inference from data", 0) \
|
||||
\
|
||||
M(UInt64, http_max_tries, 10, "Max attempts to read via http.", 0) \
|
||||
|
@ -118,7 +118,11 @@ void CachedOnDiskReadBufferFromFile::initialize(size_t offset, size_t size)
|
||||
}
|
||||
else
|
||||
{
|
||||
file_segments_holder.emplace(cache->getOrSet(cache_key, offset, size, is_persistent));
|
||||
CreateFileSegmentSettings create_settings{
|
||||
.is_persistent = is_persistent
|
||||
};
|
||||
|
||||
file_segments_holder.emplace(cache->getOrSet(cache_key, offset, size, create_settings));
|
||||
}
|
||||
|
||||
/**
|
||||
@ -156,7 +160,7 @@ CachedOnDiskReadBufferFromFile::getCacheReadBuffer(size_t offset) const
|
||||
}
|
||||
|
||||
CachedOnDiskReadBufferFromFile::ImplementationBufferPtr
|
||||
CachedOnDiskReadBufferFromFile::getRemoteFSReadBuffer(FileSegmentPtr & file_segment, ReadType read_type_)
|
||||
CachedOnDiskReadBufferFromFile::getRemoteFSReadBuffer(FileSegment & file_segment, ReadType read_type_)
|
||||
{
|
||||
switch (read_type_)
|
||||
{
|
||||
@ -178,7 +182,7 @@ CachedOnDiskReadBufferFromFile::getRemoteFSReadBuffer(FileSegmentPtr & file_segm
|
||||
* Implementation buffer from segment1 is passed to segment2 once segment1 is loaded.
|
||||
*/
|
||||
|
||||
auto remote_fs_segment_reader = file_segment->getRemoteFileReader();
|
||||
auto remote_fs_segment_reader = file_segment.getRemoteFileReader();
|
||||
|
||||
if (!remote_fs_segment_reader)
|
||||
{
|
||||
@ -189,7 +193,7 @@ CachedOnDiskReadBufferFromFile::getRemoteFSReadBuffer(FileSegmentPtr & file_segm
|
||||
ErrorCodes::CANNOT_USE_CACHE,
|
||||
"Cache cannot be used with a ReadBuffer which does not support right bounded reads");
|
||||
|
||||
file_segment->setRemoteFileReader(remote_fs_segment_reader);
|
||||
file_segment.setRemoteFileReader(remote_fs_segment_reader);
|
||||
}
|
||||
|
||||
return remote_fs_segment_reader;
|
||||
@ -201,8 +205,8 @@ CachedOnDiskReadBufferFromFile::getRemoteFSReadBuffer(FileSegmentPtr & file_segm
|
||||
if (remote_file_reader && remote_file_reader->getFileOffsetOfBufferEnd() == file_offset_of_buffer_end)
|
||||
return remote_file_reader;
|
||||
|
||||
auto remote_fs_segment_reader = file_segment->extractRemoteFileReader();
|
||||
if (remote_fs_segment_reader)
|
||||
auto remote_fs_segment_reader = file_segment.extractRemoteFileReader();
|
||||
if (remote_fs_segment_reader && file_offset_of_buffer_end == remote_file_reader->getFileOffsetOfBufferEnd())
|
||||
remote_file_reader = remote_fs_segment_reader;
|
||||
else
|
||||
remote_file_reader = implementation_buffer_creator();
|
||||
@ -217,6 +221,19 @@ CachedOnDiskReadBufferFromFile::getRemoteFSReadBuffer(FileSegmentPtr & file_segm
|
||||
}
|
||||
}
|
||||
|
||||
bool CachedOnDiskReadBufferFromFile::canStartFromCache(size_t current_offset, const FileSegment & file_segment)
|
||||
{
|
||||
/// segment{k} state: DOWNLOADING
|
||||
/// cache: [______|___________
|
||||
/// ^
|
||||
/// first_non_downloaded_offset (in progress)
|
||||
/// requested_range: [__________]
|
||||
/// ^
|
||||
/// current_offset
|
||||
size_t first_non_downloaded_offset = file_segment.getFirstNonDownloadedOffset();
|
||||
return first_non_downloaded_offset > current_offset;
|
||||
}
|
||||
|
||||
CachedOnDiskReadBufferFromFile::ImplementationBufferPtr
|
||||
CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & file_segment)
|
||||
{
|
||||
@ -236,7 +253,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
|
||||
{
|
||||
LOG_DEBUG(log, "Bypassing cache because `read_from_filesystem_cache_if_exists_otherwise_bypass_cache` option is used");
|
||||
read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE;
|
||||
return getRemoteFSReadBuffer(file_segment, read_type);
|
||||
return getRemoteFSReadBuffer(*file_segment, read_type);
|
||||
}
|
||||
}
|
||||
|
||||
@ -248,21 +265,16 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
|
||||
{
|
||||
LOG_DEBUG(log, "Bypassing cache because file segment state is `SKIP_CACHE`");
|
||||
read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE;
|
||||
return getRemoteFSReadBuffer(file_segment, read_type);
|
||||
return getRemoteFSReadBuffer(*file_segment, read_type);
|
||||
}
|
||||
case FileSegment::State::DOWNLOADING:
|
||||
{
|
||||
size_t download_offset = file_segment->getDownloadOffset();
|
||||
bool can_start_from_cache = download_offset > file_offset_of_buffer_end;
|
||||
|
||||
/// If file segment is being downloaded but we can already read
|
||||
/// from already downloaded part, do that.
|
||||
if (can_start_from_cache)
|
||||
if (canStartFromCache(file_offset_of_buffer_end, *file_segment))
|
||||
{
|
||||
/// segment{k} state: DOWNLOADING
|
||||
/// cache: [______|___________
|
||||
/// ^
|
||||
/// download_offset (in progress)
|
||||
/// first_non_downloaded_offset (in progress)
|
||||
/// requested_range: [__________]
|
||||
/// ^
|
||||
/// file_offset_of_buffer_end
|
||||
@ -282,12 +294,12 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
|
||||
case FileSegment::State::EMPTY:
|
||||
case FileSegment::State::PARTIALLY_DOWNLOADED:
|
||||
{
|
||||
if (file_segment->getDownloadOffset() > file_offset_of_buffer_end)
|
||||
if (canStartFromCache(file_offset_of_buffer_end, *file_segment))
|
||||
{
|
||||
/// segment{k} state: PARTIALLY_DOWNLOADED
|
||||
/// cache: [______|___________
|
||||
/// ^
|
||||
/// download_offset (in progress)
|
||||
/// first_non_downloaded_offset (in progress)
|
||||
/// requested_range: [__________]
|
||||
/// ^
|
||||
/// file_offset_of_buffer_end
|
||||
@ -299,20 +311,12 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
|
||||
auto downloader_id = file_segment->getOrSetDownloader();
|
||||
if (downloader_id == file_segment->getCallerId())
|
||||
{
|
||||
size_t download_offset = file_segment->getDownloadOffset();
|
||||
bool can_start_from_cache = download_offset > file_offset_of_buffer_end;
|
||||
|
||||
LOG_TEST(
|
||||
log,
|
||||
"Current download offset: {}, file offset of buffer end: {}",
|
||||
download_offset, file_offset_of_buffer_end);
|
||||
|
||||
if (can_start_from_cache)
|
||||
if (canStartFromCache(file_offset_of_buffer_end, *file_segment))
|
||||
{
|
||||
/// segment{k}
|
||||
/// cache: [______|___________
|
||||
/// ^
|
||||
/// download_offset
|
||||
/// first_non_downloaded_offset
|
||||
/// requested_range: [__________]
|
||||
/// ^
|
||||
/// file_offset_of_buffer_end
|
||||
@ -322,27 +326,24 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
|
||||
return getCacheReadBuffer(range.left);
|
||||
}
|
||||
|
||||
if (download_offset < file_offset_of_buffer_end)
|
||||
if (file_segment->getCurrentWriteOffset() < file_offset_of_buffer_end)
|
||||
{
|
||||
/// segment{1}
|
||||
/// cache: [_____|___________
|
||||
/// ^
|
||||
/// download_offset
|
||||
/// current_write_offset
|
||||
/// requested_range: [__________]
|
||||
/// ^
|
||||
/// file_offset_of_buffer_end
|
||||
|
||||
assert(file_offset_of_buffer_end > file_segment->getDownloadOffset());
|
||||
bytes_to_predownload = file_offset_of_buffer_end - file_segment->getDownloadOffset();
|
||||
assert(bytes_to_predownload < range.size());
|
||||
LOG_TEST(log, "Predownload. File segment info: {}", file_segment->getInfoForLog());
|
||||
chassert(file_offset_of_buffer_end > file_segment->getCurrentWriteOffset());
|
||||
bytes_to_predownload = file_offset_of_buffer_end - file_segment->getCurrentWriteOffset();
|
||||
chassert(bytes_to_predownload < range.size());
|
||||
}
|
||||
|
||||
download_offset = file_segment->getDownloadOffset();
|
||||
can_start_from_cache = download_offset > file_offset_of_buffer_end;
|
||||
assert(!can_start_from_cache);
|
||||
|
||||
read_type = ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE;
|
||||
return getRemoteFSReadBuffer(file_segment, read_type);
|
||||
return getRemoteFSReadBuffer(*file_segment, read_type);
|
||||
}
|
||||
|
||||
download_state = file_segment->state();
|
||||
@ -350,10 +351,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
|
||||
}
|
||||
case FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION:
|
||||
{
|
||||
size_t download_offset = file_segment->getDownloadOffset();
|
||||
bool can_start_from_cache = download_offset > file_offset_of_buffer_end;
|
||||
|
||||
if (can_start_from_cache)
|
||||
if (canStartFromCache(file_offset_of_buffer_end, *file_segment))
|
||||
{
|
||||
read_type = ReadType::CACHED;
|
||||
return getCacheReadBuffer(range.left);
|
||||
@ -364,7 +362,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
|
||||
log,
|
||||
"Bypassing cache because file segment state is `PARTIALLY_DOWNLOADED_NO_CONTINUATION` and downloaded part already used");
|
||||
read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE;
|
||||
return getRemoteFSReadBuffer(file_segment, read_type);
|
||||
return getRemoteFSReadBuffer(*file_segment, read_type);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -374,8 +372,8 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
|
||||
CachedOnDiskReadBufferFromFile::ImplementationBufferPtr
|
||||
CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_segment)
|
||||
{
|
||||
assert(!file_segment->isDownloader());
|
||||
assert(file_offset_of_buffer_end >= file_segment->range().left);
|
||||
chassert(!file_segment->isDownloader());
|
||||
chassert(file_offset_of_buffer_end >= file_segment->range().left);
|
||||
|
||||
auto range = file_segment->range();
|
||||
bytes_to_predownload = 0;
|
||||
@ -389,10 +387,10 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_se
|
||||
ProfileEvents::FileSegmentWaitReadBufferMicroseconds, watch.elapsedMicroseconds());
|
||||
|
||||
[[maybe_unused]] auto download_current_segment = read_type == ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE;
|
||||
assert(download_current_segment == file_segment->isDownloader());
|
||||
chassert(download_current_segment == file_segment->isDownloader());
|
||||
|
||||
assert(file_segment->range() == range);
|
||||
assert(file_offset_of_buffer_end >= range.left && file_offset_of_buffer_end <= range.right);
|
||||
chassert(file_segment->range() == range);
|
||||
chassert(file_offset_of_buffer_end >= range.left && file_offset_of_buffer_end <= range.right);
|
||||
|
||||
LOG_TEST(
|
||||
log,
|
||||
@ -441,12 +439,12 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_se
|
||||
}
|
||||
case ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE:
|
||||
{
|
||||
assert(file_segment->isDownloader());
|
||||
chassert(file_segment->isDownloader());
|
||||
|
||||
if (bytes_to_predownload)
|
||||
{
|
||||
size_t download_offset = file_segment->getDownloadOffset();
|
||||
read_buffer_for_file_segment->seek(download_offset, SEEK_SET);
|
||||
size_t current_write_offset = file_segment->getCurrentWriteOffset();
|
||||
read_buffer_for_file_segment->seek(current_write_offset, SEEK_SET);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -456,18 +454,15 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_se
|
||||
assert(static_cast<size_t>(read_buffer_for_file_segment->getFileOffsetOfBufferEnd()) == file_offset_of_buffer_end);
|
||||
}
|
||||
|
||||
auto download_offset = file_segment->getDownloadOffset();
|
||||
if (download_offset != static_cast<size_t>(read_buffer_for_file_segment->getPosition()))
|
||||
auto current_write_offset = file_segment->getCurrentWriteOffset();
|
||||
if (current_write_offset != static_cast<size_t>(read_buffer_for_file_segment->getPosition()))
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Buffer's offsets mismatch; cached buffer offset: {}, download_offset: {}, "
|
||||
"position: {}, implementation buffer remaining read range: {}, file segment info: {}",
|
||||
file_offset_of_buffer_end,
|
||||
download_offset,
|
||||
read_buffer_for_file_segment->getPosition(),
|
||||
read_buffer_for_file_segment->getRemainingReadRange().toString(),
|
||||
file_segment->getInfoForLog());
|
||||
"Buffer's offsets mismatch. Cached buffer offset: {}, current_write_offset: {} implementation buffer offset: {}, "
|
||||
"implementation buffer remaining range: {}, file segment info: {}",
|
||||
file_offset_of_buffer_end, current_write_offset, read_buffer_for_file_segment->getPosition(),
|
||||
read_buffer_for_file_segment->getRemainingReadRange().toString(), file_segment->getInfoForLog());
|
||||
}
|
||||
|
||||
break;
|
||||
@ -488,7 +483,7 @@ bool CachedOnDiskReadBufferFromFile::completeFileSegmentAndGetNext()
|
||||
auto & file_segment = *file_segment_it;
|
||||
|
||||
[[maybe_unused]] const auto & range = file_segment->range();
|
||||
assert(file_offset_of_buffer_end > range.right);
|
||||
chassert(file_offset_of_buffer_end > range.right);
|
||||
|
||||
LOG_TEST(
|
||||
log,
|
||||
@ -499,10 +494,8 @@ bool CachedOnDiskReadBufferFromFile::completeFileSegmentAndGetNext()
|
||||
|
||||
/// Do not hold pointer to file segment if it is not needed anymore
|
||||
/// so can become releasable and can be evicted from cache.
|
||||
/// If the status of filesegment state is SKIP_CACHE, it will not be deleted.
|
||||
/// It will be deleted from the cache when the holder is destructed.
|
||||
if ((*file_segment_it)->state() != FileSegment::State::SKIP_CACHE)
|
||||
file_segments_holder->file_segments.erase(file_segment_it);
|
||||
file_segment->completeWithoutState();
|
||||
file_segments_holder->file_segments.erase(file_segment_it);
|
||||
|
||||
if (current_file_segment_it == file_segments_holder->file_segments.end())
|
||||
return false;
|
||||
@ -545,8 +538,8 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment)
|
||||
/// download from offset a'' < a', but return buffer from offset a'.
|
||||
LOG_TEST(log, "Bytes to predownload: {}, caller_id: {}", bytes_to_predownload, FileSegment::getCallerId());
|
||||
|
||||
assert(implementation_buffer->getFileOffsetOfBufferEnd() == file_segment->getDownloadOffset());
|
||||
size_t current_offset = file_segment->getDownloadOffset();
|
||||
chassert(implementation_buffer->getFileOffsetOfBufferEnd() == file_segment->getCurrentWriteOffset());
|
||||
size_t current_offset = file_segment->getCurrentWriteOffset();
|
||||
const auto & current_range = file_segment->range();
|
||||
|
||||
while (true)
|
||||
@ -572,7 +565,7 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment)
|
||||
"current download offset: {}, expected: {}, eof: {}",
|
||||
bytes_to_predownload,
|
||||
current_range.toString(),
|
||||
file_segment->getDownloadOffset(),
|
||||
file_segment->getCurrentWriteOffset(),
|
||||
file_offset_of_buffer_end,
|
||||
implementation_buffer->eof());
|
||||
|
||||
@ -582,18 +575,20 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment)
|
||||
{
|
||||
nextimpl_working_buffer_offset = implementation_buffer->offset();
|
||||
|
||||
auto download_offset = file_segment->getDownloadOffset();
|
||||
if (download_offset != static_cast<size_t>(implementation_buffer->getPosition())
|
||||
|| download_offset != file_offset_of_buffer_end)
|
||||
auto current_write_offset = file_segment->getCurrentWriteOffset();
|
||||
if (current_write_offset != static_cast<size_t>(implementation_buffer->getPosition())
|
||||
|| current_write_offset != file_offset_of_buffer_end)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Buffer's offsets mismatch after predownloading; download offset: {}, "
|
||||
"cached buffer offset: {}, implementation buffer offset: {}, "
|
||||
"file segment info: {}",
|
||||
download_offset,
|
||||
current_write_offset,
|
||||
file_offset_of_buffer_end,
|
||||
implementation_buffer->getPosition(),
|
||||
file_segment->getInfoForLog());
|
||||
}
|
||||
}
|
||||
|
||||
break;
|
||||
@ -609,7 +604,7 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment)
|
||||
{
|
||||
LOG_TEST(log, "Left to predownload: {}, buffer size: {}", bytes_to_predownload, current_impl_buffer_size);
|
||||
|
||||
assert(file_segment->getDownloadOffset() == static_cast<size_t>(implementation_buffer->getPosition()));
|
||||
chassert(file_segment->getCurrentWriteOffset() == static_cast<size_t>(implementation_buffer->getPosition()));
|
||||
|
||||
bool success = writeCache(implementation_buffer->buffer().begin(), current_predownload_size, current_offset, *file_segment);
|
||||
if (success)
|
||||
@ -635,7 +630,7 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment)
|
||||
/// segment{1}
|
||||
/// cache: [_____|___________
|
||||
/// ^
|
||||
/// download_offset
|
||||
/// current_write_offset
|
||||
/// requested_range: [__________]
|
||||
/// ^
|
||||
/// file_offset_of_buffer_end
|
||||
@ -649,17 +644,18 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment)
|
||||
bytes_to_predownload = 0;
|
||||
file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION);
|
||||
|
||||
LOG_TEST(log, "Bypassing cache because space reservation failed");
|
||||
LOG_TEST(log, "Bypassing cache because for {}", file_segment->getInfoForLog());
|
||||
|
||||
read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE;
|
||||
|
||||
swap(*implementation_buffer);
|
||||
resetWorkingBuffer();
|
||||
|
||||
implementation_buffer = getRemoteFSReadBuffer(file_segment, read_type);
|
||||
implementation_buffer = getRemoteFSReadBuffer(*file_segment, read_type);
|
||||
|
||||
swap(*implementation_buffer);
|
||||
|
||||
implementation_buffer->setReadUntilPosition(current_range.right + 1); /// [..., range.right]
|
||||
implementation_buffer->setReadUntilPosition(file_segment->range().right + 1); /// [..., range.right]
|
||||
implementation_buffer->seek(file_offset_of_buffer_end, SEEK_SET);
|
||||
|
||||
LOG_TEST(
|
||||
@ -680,8 +676,8 @@ bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded()
|
||||
auto current_read_range = file_segment->range();
|
||||
auto current_state = file_segment->state();
|
||||
|
||||
assert(current_read_range.left <= file_offset_of_buffer_end);
|
||||
assert(!file_segment->isDownloader());
|
||||
chassert(current_read_range.left <= file_offset_of_buffer_end);
|
||||
chassert(!file_segment->isDownloader());
|
||||
|
||||
if (file_offset_of_buffer_end > current_read_range.right)
|
||||
{
|
||||
@ -695,13 +691,15 @@ bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded()
|
||||
/// segment{k}
|
||||
/// cache: [______|___________
|
||||
/// ^
|
||||
/// download_offset
|
||||
/// current_write_offset
|
||||
/// requested_range: [__________]
|
||||
/// ^
|
||||
/// file_offset_of_buffer_end
|
||||
|
||||
size_t download_offset = file_segment->getDownloadOffset();
|
||||
bool cached_part_is_finished = download_offset == file_offset_of_buffer_end;
|
||||
auto current_write_offset = file_segment->getCurrentWriteOffset();
|
||||
bool cached_part_is_finished = current_write_offset == file_offset_of_buffer_end;
|
||||
|
||||
LOG_TEST(log, "Current write offset: {}, file offset of buffer end: {}", current_write_offset, file_offset_of_buffer_end);
|
||||
|
||||
if (cached_part_is_finished)
|
||||
{
|
||||
@ -710,12 +708,12 @@ bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded()
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (download_offset < file_offset_of_buffer_end)
|
||||
else if (current_write_offset < file_offset_of_buffer_end)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Expected {} >= {} ({})",
|
||||
download_offset, file_offset_of_buffer_end, getInfoForLog());
|
||||
current_write_offset, file_offset_of_buffer_end, getInfoForLog());
|
||||
}
|
||||
}
|
||||
|
||||
@ -725,7 +723,7 @@ bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded()
|
||||
* ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE means that on previous getImplementationBuffer() call
|
||||
* current buffer successfully called file_segment->getOrSetDownloader() and became a downloader
|
||||
* for this file segment. However, the downloader's term has a lifespan of 1 nextImpl() call,
|
||||
* e.g. downloader reads buffer_size byte and calls completeBatchAndResetDownloader() and some other
|
||||
* e.g. downloader reads buffer_size byte and calls completePartAndResetDownloader() and some other
|
||||
* thread can become a downloader if it calls getOrSetDownloader() faster.
|
||||
*
|
||||
* So downloader is committed to download only buffer_size bytes and then is not a downloader anymore,
|
||||
@ -817,11 +815,11 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
|
||||
if (need_complete_file_segment)
|
||||
{
|
||||
LOG_TEST(log, "Resetting downloader {} from scope exit", file_segment->getDownloader());
|
||||
file_segment->completeBatchAndResetDownloader();
|
||||
file_segment->completePartAndResetDownloader();
|
||||
}
|
||||
}
|
||||
|
||||
assert(!file_segment->isDownloader());
|
||||
chassert(!file_segment->isDownloader());
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -845,7 +843,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
|
||||
(*current_file_segment_it)->incrementHitsCount();
|
||||
}
|
||||
|
||||
assert(!internal_buffer.empty());
|
||||
chassert(!internal_buffer.empty());
|
||||
|
||||
swap(*implementation_buffer);
|
||||
|
||||
@ -854,15 +852,14 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
|
||||
|
||||
LOG_TEST(
|
||||
log,
|
||||
"Current segment: {}, downloader: {}, current count: {}, position: {}, read range: {}",
|
||||
current_read_range.toString(),
|
||||
file_segment->getDownloader(),
|
||||
"Current count: {}, position: {}, read range: {}, file segment: {}",
|
||||
implementation_buffer->count(),
|
||||
implementation_buffer->getPosition(),
|
||||
implementation_buffer->getRemainingReadRange().toString());
|
||||
implementation_buffer->getRemainingReadRange().toString(),
|
||||
file_segment->getInfoForLog());
|
||||
|
||||
assert(current_read_range.left <= file_offset_of_buffer_end);
|
||||
assert(current_read_range.right >= file_offset_of_buffer_end);
|
||||
chassert(current_read_range.left <= file_offset_of_buffer_end);
|
||||
chassert(current_read_range.right >= file_offset_of_buffer_end);
|
||||
|
||||
bool result = false;
|
||||
size_t size = 0;
|
||||
@ -939,24 +936,26 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
|
||||
{
|
||||
if (download_current_segment)
|
||||
{
|
||||
assert(file_offset_of_buffer_end + size - 1 <= file_segment->range().right);
|
||||
chassert(file_offset_of_buffer_end + size - 1 <= file_segment->range().right);
|
||||
|
||||
bool success = file_segment->reserve(size);
|
||||
if (success)
|
||||
{
|
||||
assert(file_segment->getDownloadOffset() == static_cast<size_t>(implementation_buffer->getPosition()));
|
||||
chassert(file_segment->getCurrentWriteOffset() == static_cast<size_t>(implementation_buffer->getPosition()));
|
||||
|
||||
success = writeCache(implementation_buffer->position(), size, file_offset_of_buffer_end, *file_segment);
|
||||
if (success)
|
||||
{
|
||||
assert(file_segment->getDownloadOffset() <= file_segment->range().right + 1);
|
||||
assert(
|
||||
chassert(file_segment->getCurrentWriteOffset() <= file_segment->range().right + 1);
|
||||
chassert(
|
||||
std::next(current_file_segment_it) == file_segments_holder->file_segments.end()
|
||||
|| file_segment->getDownloadOffset() == implementation_buffer->getFileOffsetOfBufferEnd());
|
||||
|| file_segment->getCurrentWriteOffset() == implementation_buffer->getFileOffsetOfBufferEnd());
|
||||
|
||||
LOG_TEST(log, "Successfully written {} bytes", size);
|
||||
}
|
||||
else
|
||||
{
|
||||
assert(file_segment->state() == FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION);
|
||||
chassert(file_segment->state() == FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION);
|
||||
LOG_TEST(log, "Bypassing cache because writeCache method failed");
|
||||
}
|
||||
}
|
||||
@ -984,7 +983,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
|
||||
size_t remaining_size_to_read
|
||||
= std::min(current_read_range.right, read_until_position - 1) - file_offset_of_buffer_end + 1;
|
||||
size = std::min(size, remaining_size_to_read);
|
||||
assert(implementation_buffer->buffer().size() >= nextimpl_working_buffer_offset + size);
|
||||
chassert(implementation_buffer->buffer().size() >= nextimpl_working_buffer_offset + size);
|
||||
implementation_buffer->buffer().resize(nextimpl_working_buffer_offset + size);
|
||||
}
|
||||
|
||||
@ -996,15 +995,15 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
|
||||
current_file_segment_counters.increment(ProfileEvents::FileSegmentUsedBytes, available());
|
||||
|
||||
if (download_current_segment)
|
||||
file_segment->completeBatchAndResetDownloader();
|
||||
file_segment->completePartAndResetDownloader();
|
||||
|
||||
assert(!file_segment->isDownloader());
|
||||
chassert(!file_segment->isDownloader());
|
||||
|
||||
LOG_TEST(
|
||||
log,
|
||||
"Key: {}. Returning with {} bytes, buffer position: {} (offset: {}, predownloaded: {}), "
|
||||
"buffer available: {}, current range: {}, current offset: {}, file segment state: {}, "
|
||||
"download offset: {}, read_type: {}, reading until position: {}, started with offset: {}, "
|
||||
"current write offset: {}, read_type: {}, reading until position: {}, started with offset: {}, "
|
||||
"remaining ranges: {}",
|
||||
getHexUIntLowercase(cache_key),
|
||||
working_buffer.size(),
|
||||
@ -1015,7 +1014,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
|
||||
current_read_range.toString(),
|
||||
file_offset_of_buffer_end,
|
||||
FileSegment::stateToString(file_segment->state()),
|
||||
file_segment->getDownloadOffset(),
|
||||
file_segment->getCurrentWriteOffset(),
|
||||
toString(read_type),
|
||||
read_until_position,
|
||||
first_offset,
|
||||
|
@ -80,7 +80,7 @@ private:
|
||||
|
||||
void assertCorrectness() const;
|
||||
|
||||
std::shared_ptr<ReadBufferFromFileBase> getRemoteFSReadBuffer(FileSegmentPtr & file_segment, ReadType read_type_);
|
||||
std::shared_ptr<ReadBufferFromFileBase> getRemoteFSReadBuffer(FileSegment & file_segment, ReadType read_type_);
|
||||
|
||||
size_t getTotalSizeToRead();
|
||||
|
||||
@ -90,6 +90,8 @@ private:
|
||||
|
||||
bool writeCache(char * data, size_t size, size_t offset, FileSegment & file_segment);
|
||||
|
||||
static bool canStartFromCache(size_t current_offset, const FileSegment & file_segment);
|
||||
|
||||
Poco::Logger * log;
|
||||
FileCache::Key cache_key;
|
||||
String source_file_path;
|
||||
|
@ -77,7 +77,7 @@ void CachedOnDiskWriteBufferFromFile::nextImpl()
|
||||
|
||||
void CachedOnDiskWriteBufferFromFile::cacheData(char * data, size_t size)
|
||||
{
|
||||
if (stop_caching)
|
||||
if (cache_in_error_state_or_disabled)
|
||||
return;
|
||||
|
||||
if (!cache_writer)
|
||||
@ -88,14 +88,13 @@ void CachedOnDiskWriteBufferFromFile::cacheData(char * data, size_t size)
|
||||
|
||||
Stopwatch watch(CLOCK_MONOTONIC);
|
||||
|
||||
cache_in_error_state_or_disabled = true;
|
||||
|
||||
try
|
||||
{
|
||||
if (!cache_writer->write(data, size, current_download_offset, is_persistent_cache_file))
|
||||
{
|
||||
LOG_INFO(log, "Write-through cache is stopped as cache limit is reached and nothing can be evicted");
|
||||
|
||||
/// No space left, disable caching.
|
||||
stop_caching = true;
|
||||
return;
|
||||
}
|
||||
}
|
||||
@ -122,6 +121,8 @@ void CachedOnDiskWriteBufferFromFile::cacheData(char * data, size_t size)
|
||||
|
||||
current_file_segment_counters.increment(
|
||||
ProfileEvents::FileSegmentWriteMicroseconds, watch.elapsedMicroseconds());
|
||||
|
||||
cache_in_error_state_or_disabled = false;
|
||||
}
|
||||
|
||||
void CachedOnDiskWriteBufferFromFile::appendFilesystemCacheLog(const FileSegment & file_segment)
|
||||
|
@ -51,7 +51,7 @@ private:
|
||||
bool enable_cache_log;
|
||||
std::shared_ptr<FilesystemCacheLog> cache_log;
|
||||
|
||||
bool stop_caching = false;
|
||||
bool cache_in_error_state_or_disabled = false;
|
||||
|
||||
ProfileEvents::Counters current_file_segment_counters;
|
||||
std::unique_ptr<FileSegmentRangeWriter> cache_writer;
|
||||
|
@ -76,6 +76,8 @@ struct ReadSettings
|
||||
/// For 'pread_threadpool' method. Lower is more priority.
|
||||
size_t priority = 0;
|
||||
|
||||
bool load_marks_asynchronously = true;
|
||||
|
||||
size_t remote_fs_read_max_backoff_ms = 10000;
|
||||
size_t remote_fs_read_backoff_max_tries = 4;
|
||||
|
||||
|
@ -26,6 +26,7 @@
|
||||
#include <Common/config.h>
|
||||
#include <Common/config_version.h>
|
||||
|
||||
#include <filesystem>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
@ -346,13 +347,29 @@ namespace detail
|
||||
non_retriable_errors.begin(), non_retriable_errors.end(), [&](const auto status) { return http_status != status; });
|
||||
}
|
||||
|
||||
Poco::URI getUriAfterRedirect(const Poco::URI & prev_uri, Poco::Net::HTTPResponse & response)
|
||||
{
|
||||
auto location = response.get("Location");
|
||||
auto location_uri = Poco::URI(location);
|
||||
if (!location_uri.isRelative())
|
||||
return location_uri;
|
||||
/// Location header contains relative path. So we need to concatenate it
|
||||
/// with path from the original URI and normalize it.
|
||||
auto path = std::filesystem::weakly_canonical(std::filesystem::path(prev_uri.getPath()) / location);
|
||||
location_uri = prev_uri;
|
||||
location_uri.setPath(path);
|
||||
return location_uri;
|
||||
}
|
||||
|
||||
void callWithRedirects(Poco::Net::HTTPResponse & response, const String & method_, bool throw_on_all_errors = false)
|
||||
{
|
||||
call(response, method_, throw_on_all_errors);
|
||||
Poco::URI prev_uri = uri;
|
||||
|
||||
while (isRedirect(response.getStatus()))
|
||||
{
|
||||
Poco::URI uri_redirect(response.get("Location"));
|
||||
Poco::URI uri_redirect = getUriAfterRedirect(prev_uri, response);
|
||||
prev_uri = uri_redirect;
|
||||
if (remote_host_filter)
|
||||
remote_host_filter->checkURL(uri_redirect);
|
||||
|
||||
@ -408,7 +425,7 @@ namespace detail
|
||||
|
||||
while (isRedirect(response.getStatus()))
|
||||
{
|
||||
Poco::URI uri_redirect(response.get("Location"));
|
||||
Poco::URI uri_redirect = getUriAfterRedirect(saved_uri_redirect.value_or(uri), response);
|
||||
if (remote_host_filter)
|
||||
remote_host_filter->checkURL(uri_redirect);
|
||||
|
||||
|
@ -141,7 +141,13 @@ AsynchronousInsertQueue::~AsynchronousInsertQueue()
|
||||
{
|
||||
/// TODO: add a setting for graceful shutdown.
|
||||
|
||||
shutdown = true;
|
||||
LOG_TRACE(log, "Shutting down the asynchronous insertion queue");
|
||||
|
||||
{
|
||||
std::lock_guard lock(shutdown_mutex);
|
||||
shutdown = true;
|
||||
shutdown_cv.notify_all();
|
||||
}
|
||||
|
||||
assert(dump_by_first_update_thread.joinable());
|
||||
dump_by_first_update_thread.join();
|
||||
@ -162,6 +168,8 @@ AsynchronousInsertQueue::~AsynchronousInsertQueue()
|
||||
ErrorCodes::TIMEOUT_EXCEEDED,
|
||||
"Wait for async insert timeout exceeded)")));
|
||||
}
|
||||
|
||||
LOG_TRACE(log, "Asynchronous insertion queue finished");
|
||||
}
|
||||
|
||||
void AsynchronousInsertQueue::scheduleDataProcessingJob(const InsertQuery & key, InsertDataPtr data, ContextPtr global_context)
|
||||
@ -276,10 +284,8 @@ void AsynchronousInsertQueue::busyCheck()
|
||||
{
|
||||
auto timeout = busy_timeout;
|
||||
|
||||
while (!shutdown)
|
||||
while (!waitForShutdown(timeout))
|
||||
{
|
||||
std::this_thread::sleep_for(timeout);
|
||||
|
||||
/// TODO: use priority queue instead of raw unsorted queue.
|
||||
timeout = busy_timeout;
|
||||
std::shared_lock read_lock(rwlock);
|
||||
@ -301,9 +307,8 @@ void AsynchronousInsertQueue::busyCheck()
|
||||
|
||||
void AsynchronousInsertQueue::staleCheck()
|
||||
{
|
||||
while (!shutdown)
|
||||
while (!waitForShutdown(stale_timeout))
|
||||
{
|
||||
std::this_thread::sleep_for(stale_timeout);
|
||||
std::shared_lock read_lock(rwlock);
|
||||
|
||||
for (auto & [key, elem] : queue)
|
||||
@ -325,9 +330,8 @@ void AsynchronousInsertQueue::cleanup()
|
||||
/// because it holds exclusive lock.
|
||||
auto timeout = busy_timeout * 5;
|
||||
|
||||
while (!shutdown)
|
||||
while (!waitForShutdown(timeout))
|
||||
{
|
||||
std::this_thread::sleep_for(timeout);
|
||||
std::vector<InsertQuery> keys_to_remove;
|
||||
|
||||
{
|
||||
@ -379,6 +383,12 @@ void AsynchronousInsertQueue::cleanup()
|
||||
}
|
||||
}
|
||||
|
||||
bool AsynchronousInsertQueue::waitForShutdown(const Milliseconds & timeout)
|
||||
{
|
||||
std::unique_lock shutdown_lock(shutdown_mutex);
|
||||
return shutdown_cv.wait_for(shutdown_lock, timeout, [this]() { return shutdown; });
|
||||
}
|
||||
|
||||
// static
|
||||
void AsynchronousInsertQueue::processData(InsertQuery key, InsertDataPtr data, ContextPtr global_context)
|
||||
try
|
||||
|
@ -115,7 +115,10 @@ private:
|
||||
const Milliseconds busy_timeout;
|
||||
const Milliseconds stale_timeout;
|
||||
|
||||
std::atomic<bool> shutdown{false};
|
||||
std::mutex shutdown_mutex;
|
||||
std::condition_variable shutdown_cv;
|
||||
bool shutdown{false};
|
||||
|
||||
ThreadPool pool; /// dump the data only inside this pool.
|
||||
ThreadFromGlobalPool dump_by_first_update_thread; /// uses busy_timeout and busyCheck()
|
||||
ThreadFromGlobalPool dump_by_last_update_thread; /// uses stale_timeout and staleCheck()
|
||||
@ -136,6 +139,10 @@ private:
|
||||
template <typename E>
|
||||
static void finishWithException(const ASTPtr & query, const std::list<InsertData::EntryPtr> & entries, const E & exception);
|
||||
|
||||
/// @param timeout - time to wait
|
||||
/// @return true if shutdown requested
|
||||
bool waitForShutdown(const Milliseconds & timeout);
|
||||
|
||||
public:
|
||||
auto getQueueLocked() const
|
||||
{
|
||||
|
@ -122,7 +122,6 @@ void FileCache::initialize()
|
||||
fs::create_directories(cache_base_path);
|
||||
}
|
||||
|
||||
status_file = make_unique<StatusFile>(fs::path(cache_base_path) / "status", StatusFile::write_full_info);
|
||||
is_initialized = true;
|
||||
}
|
||||
}
|
||||
@ -258,7 +257,7 @@ FileSegments FileCache::splitRangeIntoCells(
|
||||
size_t offset,
|
||||
size_t size,
|
||||
FileSegment::State state,
|
||||
bool is_persistent,
|
||||
const CreateFileSegmentSettings & settings,
|
||||
std::lock_guard<std::mutex> & cache_lock)
|
||||
{
|
||||
assert(size > 0);
|
||||
@ -275,7 +274,7 @@ FileSegments FileCache::splitRangeIntoCells(
|
||||
current_cell_size = std::min(remaining_size, max_file_segment_size);
|
||||
remaining_size -= current_cell_size;
|
||||
|
||||
auto * cell = addCell(key, current_pos, current_cell_size, state, is_persistent, cache_lock);
|
||||
auto * cell = addCell(key, current_pos, current_cell_size, state, settings, cache_lock);
|
||||
if (cell)
|
||||
file_segments.push_back(cell->file_segment);
|
||||
assert(cell);
|
||||
@ -292,7 +291,7 @@ void FileCache::fillHolesWithEmptyFileSegments(
|
||||
const Key & key,
|
||||
const FileSegment::Range & range,
|
||||
bool fill_with_detached_file_segments,
|
||||
bool is_persistent,
|
||||
const CreateFileSegmentSettings & settings,
|
||||
std::lock_guard<std::mutex> & cache_lock)
|
||||
{
|
||||
/// There are segments [segment1, ..., segmentN]
|
||||
@ -339,16 +338,16 @@ void FileCache::fillHolesWithEmptyFileSegments(
|
||||
|
||||
if (fill_with_detached_file_segments)
|
||||
{
|
||||
auto file_segment = std::make_shared<FileSegment>(current_pos, hole_size, key, this, FileSegment::State::EMPTY);
|
||||
auto file_segment = std::make_shared<FileSegment>(current_pos, hole_size, key, this, FileSegment::State::EMPTY, settings);
|
||||
{
|
||||
std::lock_guard segment_lock(file_segment->mutex);
|
||||
file_segment->markAsDetached(segment_lock);
|
||||
std::unique_lock segment_lock(file_segment->mutex);
|
||||
file_segment->detachAssumeStateFinalized(segment_lock);
|
||||
}
|
||||
file_segments.insert(it, file_segment);
|
||||
}
|
||||
else
|
||||
{
|
||||
file_segments.splice(it, splitRangeIntoCells(key, current_pos, hole_size, FileSegment::State::EMPTY, is_persistent, cache_lock));
|
||||
file_segments.splice(it, splitRangeIntoCells(key, current_pos, hole_size, FileSegment::State::EMPTY, settings, cache_lock));
|
||||
}
|
||||
|
||||
current_pos = segment_range.right + 1;
|
||||
@ -366,22 +365,23 @@ void FileCache::fillHolesWithEmptyFileSegments(
|
||||
|
||||
if (fill_with_detached_file_segments)
|
||||
{
|
||||
auto file_segment = std::make_shared<FileSegment>(current_pos, hole_size, key, this, FileSegment::State::EMPTY);
|
||||
auto file_segment = std::make_shared<FileSegment>(current_pos, hole_size, key, this, FileSegment::State::EMPTY, settings);
|
||||
{
|
||||
std::lock_guard segment_lock(file_segment->mutex);
|
||||
file_segment->markAsDetached(segment_lock);
|
||||
std::unique_lock segment_lock(file_segment->mutex);
|
||||
file_segment->detachAssumeStateFinalized(segment_lock);
|
||||
}
|
||||
file_segments.insert(file_segments.end(), file_segment);
|
||||
}
|
||||
else
|
||||
{
|
||||
file_segments.splice(
|
||||
file_segments.end(), splitRangeIntoCells(key, current_pos, hole_size, FileSegment::State::EMPTY, is_persistent, cache_lock));
|
||||
file_segments.end(),
|
||||
splitRangeIntoCells(key, current_pos, hole_size, FileSegment::State::EMPTY, settings, cache_lock));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
FileSegmentsHolder FileCache::getOrSet(const Key & key, size_t offset, size_t size, bool is_persistent)
|
||||
FileSegmentsHolder FileCache::getOrSet(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings)
|
||||
{
|
||||
std::lock_guard cache_lock(mutex);
|
||||
|
||||
@ -398,11 +398,11 @@ FileSegmentsHolder FileCache::getOrSet(const Key & key, size_t offset, size_t si
|
||||
|
||||
if (file_segments.empty())
|
||||
{
|
||||
file_segments = splitRangeIntoCells(key, offset, size, FileSegment::State::EMPTY, is_persistent, cache_lock);
|
||||
file_segments = splitRangeIntoCells(key, offset, size, FileSegment::State::EMPTY, settings, cache_lock);
|
||||
}
|
||||
else
|
||||
{
|
||||
fillHolesWithEmptyFileSegments(file_segments, key, range, /* fill_with_detached */false, is_persistent, cache_lock);
|
||||
fillHolesWithEmptyFileSegments(file_segments, key, range, /* fill_with_detached */false, settings, cache_lock);
|
||||
}
|
||||
|
||||
assert(!file_segments.empty());
|
||||
@ -426,16 +426,17 @@ FileSegmentsHolder FileCache::get(const Key & key, size_t offset, size_t size)
|
||||
|
||||
if (file_segments.empty())
|
||||
{
|
||||
auto file_segment = std::make_shared<FileSegment>(offset, size, key, this, FileSegment::State::EMPTY);
|
||||
auto file_segment = std::make_shared<FileSegment>(
|
||||
offset, size, key, this, FileSegment::State::EMPTY, CreateFileSegmentSettings{});
|
||||
{
|
||||
std::lock_guard segment_lock(file_segment->mutex);
|
||||
file_segment->markAsDetached(segment_lock);
|
||||
std::unique_lock segment_lock(file_segment->mutex);
|
||||
file_segment->detachAssumeStateFinalized(segment_lock);
|
||||
}
|
||||
file_segments = { file_segment };
|
||||
}
|
||||
else
|
||||
{
|
||||
fillHolesWithEmptyFileSegments(file_segments, key, range, /* fill_with_detached */true, /* is_persistent */false, cache_lock);
|
||||
fillHolesWithEmptyFileSegments(file_segments, key, range, /* fill_with_detached */true, {}, cache_lock);
|
||||
}
|
||||
|
||||
return FileSegmentsHolder(std::move(file_segments));
|
||||
@ -443,7 +444,7 @@ FileSegmentsHolder FileCache::get(const Key & key, size_t offset, size_t size)
|
||||
|
||||
FileCache::FileSegmentCell * FileCache::addCell(
|
||||
const Key & key, size_t offset, size_t size,
|
||||
FileSegment::State state, bool is_persistent,
|
||||
FileSegment::State state, const CreateFileSegmentSettings & settings,
|
||||
std::lock_guard<std::mutex> & cache_lock)
|
||||
{
|
||||
/// Create a file segment cell and put it in `files` map by [key][offset].
|
||||
@ -475,18 +476,23 @@ FileCache::FileSegmentCell * FileCache::addCell(
|
||||
stash_records.erase({remove_priority_iter->key(), remove_priority_iter->offset()});
|
||||
remove_priority_iter->removeAndGetNext(cache_lock);
|
||||
}
|
||||
/// For segments that do not reach the download threshold, we do not download them, but directly read them
|
||||
|
||||
/// For segments that do not reach the download threshold,
|
||||
/// we do not download them, but directly read them
|
||||
result_state = FileSegment::State::SKIP_CACHE;
|
||||
}
|
||||
else
|
||||
{
|
||||
auto priority_iter = record->second;
|
||||
priority_iter->use(cache_lock);
|
||||
result_state = priority_iter->hits() >= enable_cache_hits_threshold ? FileSegment::State::EMPTY : FileSegment::State::SKIP_CACHE;
|
||||
|
||||
result_state = priority_iter->hits() >= enable_cache_hits_threshold
|
||||
? FileSegment::State::EMPTY
|
||||
: FileSegment::State::SKIP_CACHE;
|
||||
}
|
||||
}
|
||||
|
||||
return std::make_shared<FileSegment>(offset, size, key, this, result_state, is_persistent);
|
||||
return std::make_shared<FileSegment>(offset, size, key, this, result_state, settings);
|
||||
};
|
||||
|
||||
FileSegmentCell cell(skip_or_download(), this, cache_lock);
|
||||
@ -495,6 +501,7 @@ FileCache::FileSegmentCell * FileCache::addCell(
|
||||
if (offsets.empty())
|
||||
{
|
||||
auto key_path = getPathInLocalCache(key);
|
||||
|
||||
if (!fs::exists(key_path))
|
||||
fs::create_directories(key_path);
|
||||
}
|
||||
@ -513,7 +520,7 @@ FileSegmentPtr FileCache::createFileSegmentForDownload(
|
||||
const Key & key,
|
||||
size_t offset,
|
||||
size_t size,
|
||||
bool is_persistent,
|
||||
const CreateFileSegmentSettings & settings,
|
||||
std::lock_guard<std::mutex> & cache_lock)
|
||||
{
|
||||
#ifndef NDEBUG
|
||||
@ -530,7 +537,7 @@ FileSegmentPtr FileCache::createFileSegmentForDownload(
|
||||
"Cache cell already exists for key `{}` and offset {}",
|
||||
key.toString(), offset);
|
||||
|
||||
cell = addCell(key, offset, size, FileSegment::State::EMPTY, is_persistent, cache_lock);
|
||||
cell = addCell(key, offset, size, FileSegment::State::EMPTY, settings, cache_lock);
|
||||
|
||||
if (!cell)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to add a new cell for download");
|
||||
@ -542,18 +549,21 @@ bool FileCache::tryReserve(const Key & key, size_t offset, size_t size, std::loc
|
||||
{
|
||||
auto query_context = enable_filesystem_query_cache_limit ? getCurrentQueryContext(cache_lock) : nullptr;
|
||||
if (!query_context)
|
||||
{
|
||||
return tryReserveForMainList(key, offset, size, nullptr, cache_lock);
|
||||
|
||||
}
|
||||
/// The maximum cache capacity of the request is not reached, thus the
|
||||
//// cache block is evicted from the main LRU queue by tryReserveForMainList().
|
||||
else if (query_context->getCacheSize() + size <= query_context->getMaxCacheSize())
|
||||
{
|
||||
return tryReserveForMainList(key, offset, size, query_context, cache_lock);
|
||||
|
||||
}
|
||||
/// When skip_download_if_exceeds_query_cache is true, there is no need
|
||||
/// to evict old data, skip the cache and read directly from remote fs.
|
||||
else if (query_context->isSkipDownloadIfExceed())
|
||||
{
|
||||
return false;
|
||||
|
||||
}
|
||||
/// The maximum cache size of the query is reached, the cache will be
|
||||
/// evicted from the history cache accessed by the current query.
|
||||
else
|
||||
@ -833,7 +843,7 @@ void FileCache::removeIfExists(const Key & key)
|
||||
auto file_segment = cell->file_segment;
|
||||
if (file_segment)
|
||||
{
|
||||
std::lock_guard<std::mutex> segment_lock(file_segment->mutex);
|
||||
std::unique_lock<std::mutex> segment_lock(file_segment->mutex);
|
||||
file_segment->detach(cache_lock, segment_lock);
|
||||
remove(file_segment->key(), file_segment->offset(), cache_lock, segment_lock);
|
||||
}
|
||||
@ -863,9 +873,11 @@ void FileCache::removeIfReleasable()
|
||||
|
||||
auto * cell = getCell(key, offset, cache_lock);
|
||||
if (!cell)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Cache is in inconsistent state: LRU queue contains entries with no cache cell");
|
||||
}
|
||||
|
||||
if (cell->releasable())
|
||||
{
|
||||
@ -880,7 +892,7 @@ void FileCache::removeIfReleasable()
|
||||
|
||||
for (auto & file_segment : to_remove)
|
||||
{
|
||||
std::lock_guard segment_lock(file_segment->mutex);
|
||||
std::unique_lock segment_lock(file_segment->mutex);
|
||||
file_segment->detach(cache_lock, segment_lock);
|
||||
remove(file_segment->key(), file_segment->offset(), cache_lock, segment_lock);
|
||||
}
|
||||
@ -896,13 +908,13 @@ void FileCache::removeIfReleasable()
|
||||
|
||||
void FileCache::remove(FileSegmentPtr file_segment, std::lock_guard<std::mutex> & cache_lock)
|
||||
{
|
||||
std::lock_guard segment_lock(file_segment->mutex);
|
||||
std::unique_lock segment_lock(file_segment->mutex);
|
||||
remove(file_segment->key(), file_segment->offset(), cache_lock, segment_lock);
|
||||
}
|
||||
|
||||
void FileCache::remove(
|
||||
Key key, size_t offset,
|
||||
std::lock_guard<std::mutex> & cache_lock, std::lock_guard<std::mutex> & /* segment_lock */)
|
||||
std::lock_guard<std::mutex> & cache_lock, std::unique_lock<std::mutex> & /* segment_lock */)
|
||||
{
|
||||
LOG_DEBUG(log, "Remove from cache. Key: {}, offset: {}", key.toString(), offset);
|
||||
|
||||
@ -976,7 +988,7 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard<std::mutex> & cache_lock
|
||||
{
|
||||
if (!key_it->is_directory())
|
||||
{
|
||||
LOG_DEBUG(log, "Unexpected file {} (not a directory), will skip it", key_it->path().string());
|
||||
LOG_DEBUG(log, "Unexpected file: {}. Expected a directory", key_it->path().string());
|
||||
continue;
|
||||
}
|
||||
|
||||
@ -1012,7 +1024,10 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard<std::mutex> & cache_lock
|
||||
|
||||
if (tryReserve(key, offset, size, cache_lock))
|
||||
{
|
||||
auto * cell = addCell(key, offset, size, FileSegment::State::DOWNLOADED, is_persistent, cache_lock);
|
||||
auto * cell = addCell(
|
||||
key, offset, size, FileSegment::State::DOWNLOADED,
|
||||
CreateFileSegmentSettings{ .is_persistent = is_persistent }, cache_lock);
|
||||
|
||||
if (cell)
|
||||
queue_entries.emplace_back(cell->queue_iterator, cell->file_segment);
|
||||
}
|
||||
@ -1049,7 +1064,7 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard<std::mutex> & cache_lock
|
||||
|
||||
void FileCache::reduceSizeToDownloaded(
|
||||
const Key & key, size_t offset,
|
||||
std::lock_guard<std::mutex> & cache_lock, std::lock_guard<std::mutex> & /* segment_lock */)
|
||||
std::lock_guard<std::mutex> & cache_lock, std::unique_lock<std::mutex> & segment_lock)
|
||||
{
|
||||
/**
|
||||
* In case file was partially downloaded and it's download cannot be continued
|
||||
@ -1069,20 +1084,25 @@ void FileCache::reduceSizeToDownloaded(
|
||||
const auto & file_segment = cell->file_segment;
|
||||
|
||||
size_t downloaded_size = file_segment->downloaded_size;
|
||||
if (downloaded_size == file_segment->range().size())
|
||||
size_t full_size = file_segment->range().size();
|
||||
|
||||
if (downloaded_size == full_size)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Nothing to reduce, file segment fully downloaded, key: {}, offset: {}",
|
||||
key.toString(), offset);
|
||||
"Nothing to reduce, file segment fully downloaded: {}",
|
||||
file_segment->getInfoForLogUnlocked(segment_lock));
|
||||
}
|
||||
|
||||
cell->file_segment = std::make_shared<FileSegment>(offset, downloaded_size, key, this, FileSegment::State::DOWNLOADED);
|
||||
cell->file_segment = std::make_shared<FileSegment>(
|
||||
offset, downloaded_size, key, this, FileSegment::State::DOWNLOADED, CreateFileSegmentSettings{});
|
||||
|
||||
assert(file_segment->reserved_size == downloaded_size);
|
||||
}
|
||||
|
||||
bool FileCache::isLastFileSegmentHolder(
|
||||
const Key & key, size_t offset,
|
||||
std::lock_guard<std::mutex> & cache_lock, std::lock_guard<std::mutex> & /* segment_lock */)
|
||||
std::lock_guard<std::mutex> & cache_lock, std::unique_lock<std::mutex> & /* segment_lock */)
|
||||
{
|
||||
auto * cell = getCell(key, offset, cache_lock);
|
||||
|
||||
@ -1167,7 +1187,8 @@ FileCache::FileSegmentCell::FileSegmentCell(
|
||||
{
|
||||
case FileSegment::State::DOWNLOADED:
|
||||
{
|
||||
queue_iterator = cache->main_priority->add(file_segment->key(), file_segment->offset(), file_segment->range().size(), cache_lock);
|
||||
queue_iterator = cache->main_priority->add(
|
||||
file_segment->key(), file_segment->offset(), file_segment->range().size(), cache_lock);
|
||||
break;
|
||||
}
|
||||
case FileSegment::State::SKIP_CACHE:
|
||||
@ -1246,14 +1267,41 @@ void FileCache::assertPriorityCorrectness(std::lock_guard<std::mutex> & cache_lo
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Cache is in inconsistent state: LRU queue contains entries with no cache cell (assertCorrectness())");
|
||||
}
|
||||
assert(cell->size() == size);
|
||||
|
||||
if (cell->size() != size)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Expected {} == {} size ({})",
|
||||
cell->size(), size, cell->file_segment->getInfoForLog());
|
||||
}
|
||||
|
||||
total_size += size;
|
||||
}
|
||||
|
||||
assert(total_size == main_priority->getCacheSize(cache_lock));
|
||||
assert(main_priority->getCacheSize(cache_lock) <= max_size);
|
||||
assert(main_priority->getElementsNum(cache_lock) <= max_element_size);
|
||||
}
|
||||
|
||||
FileCache::QueryContextHolder::QueryContextHolder(
|
||||
const String & query_id_,
|
||||
FileCache * cache_,
|
||||
FileCache::QueryContextPtr context_)
|
||||
: query_id(query_id_)
|
||||
, cache(cache_)
|
||||
, context(context_)
|
||||
{
|
||||
}
|
||||
|
||||
FileCache::QueryContextHolder::~QueryContextHolder()
|
||||
{
|
||||
/// If only the query_map and the current holder hold the context_query,
|
||||
/// the query has been completed and the query_context is released.
|
||||
if (context && context.use_count() == 2)
|
||||
cache->removeQueryContext(query_id);
|
||||
}
|
||||
|
||||
FileCache::QueryContextPtr FileCache::getCurrentQueryContext(std::lock_guard<std::mutex> & cache_lock)
|
||||
{
|
||||
if (!isQueryInitialized())
|
||||
@ -1362,22 +1410,4 @@ void FileCache::QueryContext::use(const Key & key, size_t offset, std::lock_guar
|
||||
record->second->use(cache_lock);
|
||||
}
|
||||
|
||||
FileCache::QueryContextHolder::QueryContextHolder(
|
||||
const String & query_id_,
|
||||
FileCache * cache_,
|
||||
FileCache::QueryContextPtr context_)
|
||||
: query_id(query_id_)
|
||||
, cache(cache_)
|
||||
, context(context_)
|
||||
{
|
||||
}
|
||||
|
||||
FileCache::QueryContextHolder::~QueryContextHolder()
|
||||
{
|
||||
/// If only the query_map and the current holder hold the context_query,
|
||||
/// the query has been completed and the query_context is released.
|
||||
if (context && context.use_count() == 2)
|
||||
cache->removeQueryContext(query_id);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -12,13 +12,14 @@
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <IO/ReadSettings.h>
|
||||
#include <Interpreters/Cache/IFileCachePriority.h>
|
||||
#include <Interpreters/Cache/FileCacheKey.h>
|
||||
#include <Interpreters/Cache/FileCache_fwd.h>
|
||||
#include <Interpreters/Cache/FileSegment.h>
|
||||
#include <Interpreters/Cache/IFileCachePriority.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Interpreters/Cache/FileCacheKey.h>
|
||||
#include <Common/StatusFile.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -43,7 +44,6 @@ public:
|
||||
|
||||
~FileCache() = default;
|
||||
|
||||
/// Restore cache from local filesystem.
|
||||
void initialize();
|
||||
|
||||
const String & getBasePath() const { return cache_base_path; }
|
||||
@ -59,7 +59,7 @@ public:
|
||||
* As long as pointers to returned file segments are hold
|
||||
* it is guaranteed that these file segments are not removed from cache.
|
||||
*/
|
||||
FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size, bool is_persistent);
|
||||
FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings);
|
||||
|
||||
/**
|
||||
* Segments in returned list are ordered in ascending order and represent a full contiguous
|
||||
@ -104,7 +104,7 @@ public:
|
||||
const Key & key,
|
||||
size_t offset,
|
||||
size_t size,
|
||||
bool is_persistent,
|
||||
const CreateFileSegmentSettings & create_settings,
|
||||
std::lock_guard<std::mutex> & cache_lock);
|
||||
|
||||
FileSegments getSnapshot() const;
|
||||
@ -132,21 +132,21 @@ public:
|
||||
private:
|
||||
String cache_base_path;
|
||||
|
||||
size_t max_size;
|
||||
size_t max_element_size;
|
||||
size_t max_file_segment_size;
|
||||
const size_t max_size;
|
||||
const size_t max_element_size;
|
||||
const size_t max_file_segment_size;
|
||||
|
||||
bool allow_persistent_files;
|
||||
size_t enable_cache_hits_threshold;
|
||||
bool enable_filesystem_query_cache_limit;
|
||||
const bool allow_persistent_files;
|
||||
const size_t enable_cache_hits_threshold;
|
||||
const bool enable_filesystem_query_cache_limit;
|
||||
|
||||
mutable std::mutex mutex;
|
||||
Poco::Logger * log;
|
||||
|
||||
bool is_initialized = false;
|
||||
std::exception_ptr initialization_exception;
|
||||
std::unique_ptr<StatusFile> status_file;
|
||||
|
||||
mutable std::mutex mutex;
|
||||
void assertInitialized(std::lock_guard<std::mutex> & cache_lock) const;
|
||||
|
||||
bool tryReserve(const Key & key, size_t offset, size_t size, std::lock_guard<std::mutex> & cache_lock);
|
||||
|
||||
@ -154,7 +154,7 @@ private:
|
||||
Key key,
|
||||
size_t offset,
|
||||
std::lock_guard<std::mutex> & cache_lock,
|
||||
std::lock_guard<std::mutex> & segment_lock);
|
||||
std::unique_lock<std::mutex> & segment_lock);
|
||||
|
||||
void remove(
|
||||
FileSegmentPtr file_segment,
|
||||
@ -164,15 +164,13 @@ private:
|
||||
const Key & key,
|
||||
size_t offset,
|
||||
std::lock_guard<std::mutex> & cache_lock,
|
||||
std::lock_guard<std::mutex> & segment_lock);
|
||||
std::unique_lock<std::mutex> & segment_lock);
|
||||
|
||||
void reduceSizeToDownloaded(
|
||||
const Key & key,
|
||||
size_t offset,
|
||||
std::lock_guard<std::mutex> & cache_lock,
|
||||
std::lock_guard<std::mutex> & segment_lock);
|
||||
|
||||
void assertInitialized(std::lock_guard<std::mutex> & cache_lock) const;
|
||||
std::unique_lock<std::mutex> & segment_lock);
|
||||
|
||||
struct FileSegmentCell : private boost::noncopyable
|
||||
{
|
||||
@ -225,7 +223,7 @@ private:
|
||||
size_t offset,
|
||||
size_t size,
|
||||
FileSegment::State state,
|
||||
bool is_persistent,
|
||||
const CreateFileSegmentSettings & create_settings,
|
||||
std::lock_guard<std::mutex> & cache_lock);
|
||||
|
||||
static void useCell(const FileSegmentCell & cell, FileSegments & result, std::lock_guard<std::mutex> & cache_lock);
|
||||
@ -242,7 +240,7 @@ private:
|
||||
size_t offset,
|
||||
size_t size,
|
||||
FileSegment::State state,
|
||||
bool is_persistent,
|
||||
const CreateFileSegmentSettings & create_settings,
|
||||
std::lock_guard<std::mutex> & cache_lock);
|
||||
|
||||
String dumpStructureUnlocked(const Key & key_, std::lock_guard<std::mutex> & cache_lock);
|
||||
@ -252,7 +250,7 @@ private:
|
||||
const Key & key,
|
||||
const FileSegment::Range & range,
|
||||
bool fill_with_detached_file_segments,
|
||||
bool is_persistent,
|
||||
const CreateFileSegmentSettings & settings,
|
||||
std::lock_guard<std::mutex> & cache_lock);
|
||||
|
||||
size_t getUsedCacheSizeUnlocked(std::lock_guard<std::mutex> & cache_lock) const;
|
||||
|
@ -12,5 +12,6 @@ class FileCache;
|
||||
using FileCachePtr = std::shared_ptr<FileCache>;
|
||||
|
||||
struct FileCacheSettings;
|
||||
struct CreateFileSegmentSettings;
|
||||
|
||||
}
|
||||
|
@ -1,10 +1,10 @@
|
||||
#include "FileSegment.h"
|
||||
|
||||
#include <base/getThreadId.h>
|
||||
#include <base/scope_guard.h>
|
||||
#include <Common/scope_guard_safe.h>
|
||||
#include <Common/hex.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Interpreters/Cache/FileCache.h>
|
||||
#include <Common/hex.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <filesystem>
|
||||
@ -29,7 +29,7 @@ FileSegment::FileSegment(
|
||||
const Key & key_,
|
||||
FileCache * cache_,
|
||||
State download_state_,
|
||||
bool is_persistent_)
|
||||
const CreateFileSegmentSettings & settings)
|
||||
: segment_range(offset_, offset_ + size_ - 1)
|
||||
, download_state(download_state_)
|
||||
, file_key(key_)
|
||||
@ -39,7 +39,7 @@ FileSegment::FileSegment(
|
||||
#else
|
||||
, log(&Poco::Logger::get("FileSegment"))
|
||||
#endif
|
||||
, is_persistent(is_persistent_)
|
||||
, is_persistent(settings.is_persistent)
|
||||
{
|
||||
/// On creation, file segment state can be EMPTY, DOWNLOADED, DOWNLOADING.
|
||||
switch (download_state)
|
||||
@ -64,50 +64,79 @@ FileSegment::FileSegment(
|
||||
}
|
||||
default:
|
||||
{
|
||||
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Can create cell with either EMPTY, DOWNLOADED, DOWNLOADING state");
|
||||
throw Exception(
|
||||
ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR,
|
||||
"Can create cell with either EMPTY, DOWNLOADED, DOWNLOADING state");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
String FileSegment::getPathInLocalCache() const
|
||||
{
|
||||
return cache->getPathInLocalCache(key(), offset(), isPersistent());
|
||||
}
|
||||
|
||||
FileSegment::State FileSegment::state() const
|
||||
{
|
||||
std::lock_guard segment_lock(mutex);
|
||||
std::unique_lock segment_lock(mutex);
|
||||
return download_state;
|
||||
}
|
||||
|
||||
size_t FileSegment::getDownloadOffset() const
|
||||
void FileSegment::setDownloadState(State state)
|
||||
{
|
||||
LOG_TEST(log, "Updated state from {} to {}", stateToString(download_state), stateToString(state));
|
||||
download_state = state;
|
||||
}
|
||||
|
||||
size_t FileSegment::getFirstNonDownloadedOffset() const
|
||||
{
|
||||
std::unique_lock segment_lock(mutex);
|
||||
return getFirstNonDownloadedOffsetUnlocked(segment_lock);
|
||||
}
|
||||
|
||||
size_t FileSegment::getFirstNonDownloadedOffsetUnlocked(std::unique_lock<std::mutex> & segment_lock) const
|
||||
{
|
||||
std::lock_guard segment_lock(mutex);
|
||||
return range().left + getDownloadedSizeUnlocked(segment_lock);
|
||||
}
|
||||
|
||||
size_t FileSegment::getCurrentWriteOffset() const
|
||||
{
|
||||
std::unique_lock segment_lock(mutex);
|
||||
return getCurrentWriteOffsetUnlocked(segment_lock);
|
||||
}
|
||||
|
||||
size_t FileSegment::getCurrentWriteOffsetUnlocked(std::unique_lock<std::mutex> & segment_lock) const
|
||||
{
|
||||
return getFirstNonDownloadedOffsetUnlocked(segment_lock);
|
||||
}
|
||||
|
||||
size_t FileSegment::getDownloadedSize() const
|
||||
{
|
||||
std::lock_guard segment_lock(mutex);
|
||||
std::unique_lock segment_lock(mutex);
|
||||
return getDownloadedSizeUnlocked(segment_lock);
|
||||
}
|
||||
|
||||
size_t FileSegment::getRemainingSizeToDownload() const
|
||||
{
|
||||
std::lock_guard segment_lock(mutex);
|
||||
return range().size() - getDownloadedSizeUnlocked(segment_lock);
|
||||
}
|
||||
|
||||
bool FileSegment::isDetached() const
|
||||
{
|
||||
std::lock_guard segment_lock(mutex);
|
||||
return is_detached;
|
||||
}
|
||||
|
||||
size_t FileSegment::getDownloadedSizeUnlocked(std::lock_guard<std::mutex> & /* segment_lock */) const
|
||||
size_t FileSegment::getDownloadedSizeUnlocked(std::unique_lock<std::mutex> & /* segment_lock */) const
|
||||
{
|
||||
if (download_state == State::DOWNLOADED)
|
||||
return downloaded_size;
|
||||
|
||||
std::lock_guard download_lock(download_mutex);
|
||||
std::unique_lock download_lock(download_mutex);
|
||||
return downloaded_size;
|
||||
}
|
||||
|
||||
size_t FileSegment::getRemainingSizeToDownload() const
|
||||
{
|
||||
std::unique_lock segment_lock(mutex);
|
||||
return range().size() - getDownloadedSizeUnlocked(segment_lock);
|
||||
}
|
||||
|
||||
bool FileSegment::isDownloaded() const
|
||||
{
|
||||
std::lock_guard segment_lock(mutex);
|
||||
return is_downloaded;
|
||||
}
|
||||
|
||||
String FileSegment::getCallerId()
|
||||
{
|
||||
if (!CurrentThread::isInitialized()
|
||||
@ -118,84 +147,106 @@ String FileSegment::getCallerId()
|
||||
return std::string(CurrentThread::getQueryId()) + ":" + toString(getThreadId());
|
||||
}
|
||||
|
||||
String FileSegment::getDownloader() const
|
||||
{
|
||||
std::unique_lock segment_lock(mutex);
|
||||
return getDownloaderUnlocked(segment_lock);
|
||||
}
|
||||
|
||||
String FileSegment::getDownloaderUnlocked(std::unique_lock<std::mutex> & /* segment_lock */) const
|
||||
{
|
||||
return downloader_id;
|
||||
}
|
||||
|
||||
String FileSegment::getOrSetDownloader()
|
||||
{
|
||||
std::lock_guard segment_lock(mutex);
|
||||
std::unique_lock segment_lock(mutex);
|
||||
|
||||
assertNotDetached(segment_lock);
|
||||
assertNotDetachedUnlocked(segment_lock);
|
||||
|
||||
if (downloader_id.empty())
|
||||
auto current_downloader = getDownloaderUnlocked(segment_lock);
|
||||
|
||||
if (current_downloader.empty())
|
||||
{
|
||||
assert(download_state != State::DOWNLOADING);
|
||||
bool allow_new_downloader = download_state == State::EMPTY || download_state == State::PARTIALLY_DOWNLOADED;
|
||||
if (!allow_new_downloader)
|
||||
return "notAllowed:" + stateToString(download_state);
|
||||
|
||||
if (download_state != State::EMPTY
|
||||
&& download_state != State::PARTIALLY_DOWNLOADED)
|
||||
return "None";
|
||||
|
||||
downloader_id = getCallerId();
|
||||
download_state = State::DOWNLOADING;
|
||||
current_downloader = downloader_id = getCallerId();
|
||||
setDownloadState(State::DOWNLOADING);
|
||||
}
|
||||
else if (downloader_id == getCallerId())
|
||||
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR,
|
||||
"Attempt to set the same downloader for segment {} for the second time", range().toString());
|
||||
|
||||
return downloader_id;
|
||||
return current_downloader;
|
||||
}
|
||||
|
||||
void FileSegment::resetDownloadingStateUnlocked([[maybe_unused]] std::unique_lock<std::mutex> & segment_lock)
|
||||
{
|
||||
assert(isDownloaderUnlocked(segment_lock));
|
||||
assert(download_state == State::DOWNLOADING);
|
||||
|
||||
size_t current_downloaded_size = getDownloadedSizeUnlocked(segment_lock);
|
||||
/// range().size() can equal 0 in case of write-though cache.
|
||||
if (current_downloaded_size != 0 && current_downloaded_size == range().size())
|
||||
setDownloadedUnlocked(segment_lock);
|
||||
else
|
||||
setDownloadState(State::PARTIALLY_DOWNLOADED);
|
||||
}
|
||||
|
||||
void FileSegment::resetDownloader()
|
||||
{
|
||||
std::lock_guard segment_lock(mutex);
|
||||
std::unique_lock segment_lock(mutex);
|
||||
|
||||
assertNotDetached(segment_lock);
|
||||
assertNotDetachedUnlocked(segment_lock);
|
||||
assertIsDownloaderUnlocked("resetDownloader", segment_lock);
|
||||
|
||||
if (downloader_id.empty())
|
||||
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "There is no downloader");
|
||||
|
||||
if (getCallerId() != downloader_id)
|
||||
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Downloader can be reset only by downloader");
|
||||
|
||||
resetDownloaderImpl(segment_lock);
|
||||
resetDownloadingStateUnlocked(segment_lock);
|
||||
resetDownloaderUnlocked(segment_lock);
|
||||
}
|
||||
|
||||
void FileSegment::resetDownloaderImpl(std::lock_guard<std::mutex> & segment_lock)
|
||||
void FileSegment::resetDownloaderUnlocked(std::unique_lock<std::mutex> & /* segment_lock */)
|
||||
{
|
||||
if (getDownloadedSizeUnlocked(segment_lock) == range().size())
|
||||
setDownloaded(segment_lock);
|
||||
else
|
||||
download_state = State::PARTIALLY_DOWNLOADED;
|
||||
|
||||
LOG_TEST(log, "Resetting downloader from {}", downloader_id);
|
||||
downloader_id.clear();
|
||||
}
|
||||
|
||||
String FileSegment::getDownloader() const
|
||||
void FileSegment::assertIsDownloaderUnlocked(const std::string & operation, std::unique_lock<std::mutex> & segment_lock) const
|
||||
{
|
||||
std::lock_guard segment_lock(mutex);
|
||||
return downloader_id;
|
||||
auto caller = getCallerId();
|
||||
auto current_downloader = getDownloaderUnlocked(segment_lock);
|
||||
LOG_TEST(log, "Downloader id: {}, caller id: {}", current_downloader, caller);
|
||||
|
||||
if (caller != current_downloader)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Operation `{}` can be done only by downloader. "
|
||||
"(CallerId: {}, downloader id: {})",
|
||||
operation, caller, downloader_id);
|
||||
}
|
||||
}
|
||||
|
||||
bool FileSegment::isDownloader() const
|
||||
{
|
||||
std::lock_guard segment_lock(mutex);
|
||||
return getCallerId() == downloader_id;
|
||||
std::unique_lock segment_lock(mutex);
|
||||
return isDownloaderUnlocked(segment_lock);
|
||||
}
|
||||
|
||||
bool FileSegment::isDownloaderImpl(std::lock_guard<std::mutex> & /* segment_lock */) const
|
||||
bool FileSegment::isDownloaderUnlocked(std::unique_lock<std::mutex> & segment_lock) const
|
||||
{
|
||||
return getCallerId() == downloader_id;
|
||||
return getCallerId() == getDownloaderUnlocked(segment_lock);
|
||||
}
|
||||
|
||||
FileSegment::RemoteFileReaderPtr FileSegment::getRemoteFileReader()
|
||||
{
|
||||
if (!isDownloader())
|
||||
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Only downloader can use remote filesystem file reader");
|
||||
|
||||
std::unique_lock segment_lock(mutex);
|
||||
assertIsDownloaderUnlocked("getRemoteFileReader", segment_lock);
|
||||
return remote_file_reader;
|
||||
}
|
||||
|
||||
FileSegment::RemoteFileReaderPtr FileSegment::extractRemoteFileReader()
|
||||
{
|
||||
std::lock_guard cache_lock(cache->mutex);
|
||||
std::lock_guard segment_lock(mutex);
|
||||
std::unique_lock segment_lock(mutex);
|
||||
|
||||
if (!is_detached)
|
||||
{
|
||||
@ -210,8 +261,8 @@ FileSegment::RemoteFileReaderPtr FileSegment::extractRemoteFileReader()
|
||||
|
||||
void FileSegment::setRemoteFileReader(RemoteFileReaderPtr remote_file_reader_)
|
||||
{
|
||||
if (!isDownloader())
|
||||
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Only downloader can use remote filesystem file reader");
|
||||
std::unique_lock segment_lock(mutex);
|
||||
assertIsDownloaderUnlocked("setRemoteFileReader", segment_lock);
|
||||
|
||||
if (remote_file_reader)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Remote file reader already exists");
|
||||
@ -221,8 +272,8 @@ void FileSegment::setRemoteFileReader(RemoteFileReaderPtr remote_file_reader_)
|
||||
|
||||
void FileSegment::resetRemoteFileReader()
|
||||
{
|
||||
if (!isDownloader())
|
||||
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Only downloader can use remote filesystem file reader");
|
||||
std::unique_lock segment_lock(mutex);
|
||||
assertIsDownloaderUnlocked("resetRemoteFileReader", segment_lock);
|
||||
|
||||
if (!remote_file_reader)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Remote file reader does not exist");
|
||||
@ -230,55 +281,59 @@ void FileSegment::resetRemoteFileReader()
|
||||
remote_file_reader.reset();
|
||||
}
|
||||
|
||||
void FileSegment::write(const char * from, size_t size, size_t offset_)
|
||||
void FileSegment::write(const char * from, size_t size, size_t offset)
|
||||
{
|
||||
if (!size)
|
||||
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Writing zero size is not allowed");
|
||||
|
||||
if (availableSize() < size)
|
||||
throw Exception(
|
||||
ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR,
|
||||
"Not enough space is reserved. Available: {}, expected: {}", availableSize(), size);
|
||||
|
||||
if (!isDownloader())
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Only downloader can do the downloading. (CallerId: {}, DownloaderId: {})",
|
||||
getCallerId(), downloader_id);
|
||||
|
||||
if (getDownloadedSize() == range().size())
|
||||
throw Exception(
|
||||
ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR,
|
||||
"Attempt to write {} bytes to offset: {}, but current file segment is already fully downloaded",
|
||||
size, offset_);
|
||||
|
||||
auto download_offset = range().left + downloaded_size;
|
||||
if (offset_ != download_offset)
|
||||
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR,
|
||||
"Attempt to write {} bytes to offset: {}, but current download offset is {}",
|
||||
size, offset_, download_offset);
|
||||
|
||||
{
|
||||
std::lock_guard segment_lock(mutex);
|
||||
assertNotDetached(segment_lock);
|
||||
}
|
||||
std::unique_lock segment_lock(mutex);
|
||||
|
||||
if (!cache_writer)
|
||||
{
|
||||
if (downloaded_size > 0)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Cache writer was finalized (downloaded size: {}, state: {})",
|
||||
downloaded_size, stateToString(download_state));
|
||||
assertIsDownloaderUnlocked("write", segment_lock);
|
||||
assertNotDetachedUnlocked(segment_lock);
|
||||
|
||||
auto download_path = getPathInLocalCache();
|
||||
cache_writer = std::make_unique<WriteBufferFromFile>(download_path);
|
||||
if (download_state != State::DOWNLOADING)
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Expected DOWNLOADING state, got {}", stateToString(download_state));
|
||||
|
||||
size_t first_non_downloaded_offset = getFirstNonDownloadedOffsetUnlocked(segment_lock);
|
||||
if (offset != first_non_downloaded_offset)
|
||||
throw Exception(
|
||||
ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR,
|
||||
"Attempt to write {} bytes to offset: {}, but current write offset is {}",
|
||||
size, offset, first_non_downloaded_offset);
|
||||
|
||||
size_t current_downloaded_size = getDownloadedSizeUnlocked(segment_lock);
|
||||
chassert(reserved_size >= current_downloaded_size);
|
||||
size_t free_reserved_size = reserved_size - current_downloaded_size;
|
||||
|
||||
if (free_reserved_size < size)
|
||||
throw Exception(
|
||||
ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR,
|
||||
"Not enough space is reserved. Available: {}, expected: {}", free_reserved_size, size);
|
||||
|
||||
if (current_downloaded_size == range().size())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "File segment is already fully downloaded");
|
||||
|
||||
if (!cache_writer)
|
||||
{
|
||||
if (current_downloaded_size > 0)
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Cache writer was finalized (downloaded size: {}, state: {})",
|
||||
current_downloaded_size, stateToString(download_state));
|
||||
|
||||
auto download_path = getPathInLocalCache();
|
||||
cache_writer = std::make_unique<WriteBufferFromFile>(download_path);
|
||||
}
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
cache_writer->write(from, size);
|
||||
|
||||
std::lock_guard download_lock(download_mutex);
|
||||
std::unique_lock download_lock(download_mutex);
|
||||
|
||||
cache_writer->next();
|
||||
|
||||
@ -286,23 +341,20 @@ void FileSegment::write(const char * from, size_t size, size_t offset_)
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
std::lock_guard segment_lock(mutex);
|
||||
std::unique_lock segment_lock(mutex);
|
||||
|
||||
wrapWithCacheInfo(e, "while writing into cache", segment_lock);
|
||||
|
||||
setDownloadFailed(segment_lock);
|
||||
setDownloadFailedUnlocked(segment_lock);
|
||||
|
||||
cv.notify_all();
|
||||
|
||||
throw;
|
||||
}
|
||||
|
||||
assert(getDownloadOffset() == offset_ + size);
|
||||
}
|
||||
|
||||
String FileSegment::getPathInLocalCache() const
|
||||
{
|
||||
return cache->getPathInLocalCache(key(), offset(), isPersistent());
|
||||
#ifndef NDEBUG
|
||||
chassert(getFirstNonDownloadedOffset() == offset + size);
|
||||
#endif
|
||||
}
|
||||
|
||||
FileSegment::State FileSegment::wait()
|
||||
@ -324,8 +376,8 @@ FileSegment::State FileSegment::wait()
|
||||
{
|
||||
LOG_TEST(log, "{} waiting on: {}, current downloader: {}", getCallerId(), range().toString(), downloader_id);
|
||||
|
||||
assert(!downloader_id.empty());
|
||||
assert(downloader_id != getCallerId());
|
||||
chassert(!getDownloaderUnlocked(segment_lock).empty());
|
||||
chassert(!isDownloaderUnlocked(segment_lock));
|
||||
|
||||
cv.wait_for(segment_lock, std::chrono::seconds(60));
|
||||
}
|
||||
@ -338,30 +390,23 @@ bool FileSegment::reserve(size_t size_to_reserve)
|
||||
if (!size_to_reserve)
|
||||
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Zero space reservation is not allowed");
|
||||
|
||||
size_t expected_downloaded_size;
|
||||
|
||||
{
|
||||
std::lock_guard segment_lock(mutex);
|
||||
assertNotDetached(segment_lock);
|
||||
std::unique_lock segment_lock(mutex);
|
||||
|
||||
auto caller_id = getCallerId();
|
||||
bool is_downloader = caller_id == downloader_id;
|
||||
if (!is_downloader)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Space can be reserved only by downloader (current: {}, expected: {})",
|
||||
caller_id, downloader_id);
|
||||
}
|
||||
assertNotDetachedUnlocked(segment_lock);
|
||||
assertIsDownloaderUnlocked("reserve", segment_lock);
|
||||
|
||||
size_t current_downloaded_size = getDownloadedSizeUnlocked(segment_lock);
|
||||
if (current_downloaded_size + size_to_reserve > range().size())
|
||||
{
|
||||
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: {} ({})",
|
||||
size_to_reserve, getInfoForLogImpl(segment_lock));
|
||||
}
|
||||
"Attempt to reserve space too much space ({}) for file segment with range: {} (downloaded size: {})",
|
||||
size_to_reserve, range().toString(), downloaded_size);
|
||||
|
||||
assert(reserved_size >= current_downloaded_size);
|
||||
chassert(reserved_size >= expected_downloaded_size);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -370,9 +415,7 @@ bool FileSegment::reserve(size_t size_to_reserve)
|
||||
* and the caller is going to continue;
|
||||
*/
|
||||
|
||||
size_t current_downloaded_size = getDownloadedSize();
|
||||
assert(reserved_size >= current_downloaded_size);
|
||||
size_t already_reserved_size = reserved_size - current_downloaded_size;
|
||||
size_t already_reserved_size = reserved_size - expected_downloaded_size;
|
||||
|
||||
bool reserved = already_reserved_size >= size_to_reserve;
|
||||
if (!reserved)
|
||||
@ -392,23 +435,13 @@ bool FileSegment::reserve(size_t size_to_reserve)
|
||||
return reserved;
|
||||
}
|
||||
|
||||
bool FileSegment::isDownloaded() const
|
||||
{
|
||||
std::lock_guard segment_lock(mutex);
|
||||
return isDownloadedUnlocked(segment_lock);
|
||||
}
|
||||
|
||||
bool FileSegment::isDownloadedUnlocked(std::lock_guard<std::mutex> & /* segment_lock */) const
|
||||
{
|
||||
return is_downloaded;
|
||||
}
|
||||
|
||||
void FileSegment::setDownloaded([[maybe_unused]] std::lock_guard<std::mutex> & segment_lock)
|
||||
void FileSegment::setDownloadedUnlocked([[maybe_unused]] std::unique_lock<std::mutex> & segment_lock)
|
||||
{
|
||||
if (is_downloaded)
|
||||
return;
|
||||
|
||||
downloader_id.clear();
|
||||
setDownloadState(State::DOWNLOADED);
|
||||
is_downloaded = true;
|
||||
|
||||
if (cache_writer)
|
||||
{
|
||||
@ -424,10 +457,12 @@ void FileSegment::setDownloaded([[maybe_unused]] std::lock_guard<std::mutex> & s
|
||||
assert(std::filesystem::file_size(getPathInLocalCache()) > 0);
|
||||
}
|
||||
|
||||
void FileSegment::setDownloadFailed(std::lock_guard<std::mutex> & /* segment_lock */)
|
||||
void FileSegment::setDownloadFailedUnlocked(std::unique_lock<std::mutex> & segment_lock)
|
||||
{
|
||||
download_state = State::PARTIALLY_DOWNLOADED_NO_CONTINUATION;
|
||||
downloader_id.clear();
|
||||
LOG_INFO(log, "Settings download as failed: {}", getInfoForLogUnlocked(segment_lock));
|
||||
|
||||
setDownloadState(State::PARTIALLY_DOWNLOADED_NO_CONTINUATION);
|
||||
resetDownloaderUnlocked(segment_lock);
|
||||
|
||||
if (cache_writer)
|
||||
{
|
||||
@ -437,43 +472,31 @@ void FileSegment::setDownloadFailed(std::lock_guard<std::mutex> & /* segment_loc
|
||||
}
|
||||
}
|
||||
|
||||
void FileSegment::completeBatchAndResetDownloader()
|
||||
void FileSegment::completePartAndResetDownloader()
|
||||
{
|
||||
std::lock_guard segment_lock(mutex);
|
||||
std::unique_lock segment_lock(mutex);
|
||||
completePartAndResetDownloaderUnlocked(segment_lock);
|
||||
}
|
||||
|
||||
assertNotDetached(segment_lock);
|
||||
void FileSegment::completePartAndResetDownloaderUnlocked(std::unique_lock<std::mutex> & segment_lock)
|
||||
{
|
||||
assertNotDetachedUnlocked(segment_lock);
|
||||
assertIsDownloaderUnlocked("completePartAndResetDownloader", segment_lock);
|
||||
|
||||
if (!isDownloaderImpl(segment_lock))
|
||||
{
|
||||
cv.notify_all();
|
||||
throw Exception(
|
||||
ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR,
|
||||
"File segment can be completed only by downloader ({} != {})",
|
||||
downloader_id, getCallerId());
|
||||
}
|
||||
|
||||
resetDownloaderImpl(segment_lock);
|
||||
|
||||
LOG_TEST(log, "Complete batch. Current downloaded size: {}", getDownloadedSizeUnlocked(segment_lock));
|
||||
resetDownloadingStateUnlocked(segment_lock);
|
||||
resetDownloaderUnlocked(segment_lock);
|
||||
|
||||
LOG_TEST(log, "Complete batch. ({})", getInfoForLogUnlocked(segment_lock));
|
||||
cv.notify_all();
|
||||
}
|
||||
|
||||
void FileSegment::completeWithState(State state)
|
||||
{
|
||||
std::lock_guard cache_lock(cache->mutex);
|
||||
std::lock_guard segment_lock(mutex);
|
||||
std::unique_lock segment_lock(mutex);
|
||||
|
||||
assertNotDetached(segment_lock);
|
||||
|
||||
auto caller_id = getCallerId();
|
||||
if (caller_id != downloader_id)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"File segment completion can be done only by downloader. (CallerId: {}, downloader id: {}",
|
||||
caller_id, downloader_id);
|
||||
}
|
||||
assertNotDetachedUnlocked(segment_lock);
|
||||
assertIsDownloaderUnlocked("complete", segment_lock);
|
||||
|
||||
if (state != State::DOWNLOADED
|
||||
&& state != State::PARTIALLY_DOWNLOADED
|
||||
@ -485,24 +508,29 @@ void FileSegment::completeWithState(State state)
|
||||
"Cannot complete file segment with state: {}", stateToString(state));
|
||||
}
|
||||
|
||||
download_state = state;
|
||||
setDownloadState(state);
|
||||
completeBasedOnCurrentState(cache_lock, segment_lock);
|
||||
}
|
||||
|
||||
void FileSegment::completeWithoutState(std::lock_guard<std::mutex> & cache_lock)
|
||||
void FileSegment::completeWithoutState()
|
||||
{
|
||||
std::lock_guard segment_lock(mutex);
|
||||
std::lock_guard cache_lock(cache->mutex);
|
||||
completeWithoutStateUnlocked(cache_lock);
|
||||
}
|
||||
|
||||
void FileSegment::completeWithoutStateUnlocked(std::lock_guard<std::mutex> & cache_lock)
|
||||
{
|
||||
std::unique_lock segment_lock(mutex);
|
||||
completeBasedOnCurrentState(cache_lock, segment_lock);
|
||||
}
|
||||
|
||||
void FileSegment::completeBasedOnCurrentState(std::lock_guard<std::mutex> & cache_lock, std::lock_guard<std::mutex> & segment_lock)
|
||||
void FileSegment::completeBasedOnCurrentState(std::lock_guard<std::mutex> & cache_lock, std::unique_lock<std::mutex> & segment_lock)
|
||||
{
|
||||
if (is_detached)
|
||||
return;
|
||||
|
||||
bool is_downloader = isDownloaderImpl(segment_lock);
|
||||
bool is_downloader = isDownloaderUnlocked(segment_lock);
|
||||
bool is_last_holder = cache->isLastFileSegmentHolder(key(), offset(), cache_lock, segment_lock);
|
||||
bool can_update_segment_state = is_downloader || is_last_holder;
|
||||
size_t current_downloaded_size = getDownloadedSizeUnlocked(segment_lock);
|
||||
|
||||
SCOPE_EXIT({
|
||||
@ -512,16 +540,16 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard<std::mutex> & cach
|
||||
}
|
||||
});
|
||||
|
||||
LOG_TEST(log, "Complete without state (is_last_holder: {}). File segment info: {}", is_last_holder, getInfoForLogImpl(segment_lock));
|
||||
LOG_TEST(
|
||||
log,
|
||||
"Complete based on current state (is_last_holder: {}, {})",
|
||||
is_last_holder, getInfoForLogUnlocked(segment_lock));
|
||||
|
||||
if (can_update_segment_state)
|
||||
if (is_downloader)
|
||||
{
|
||||
if (current_downloaded_size == range().size())
|
||||
setDownloaded(segment_lock);
|
||||
else
|
||||
download_state = State::PARTIALLY_DOWNLOADED;
|
||||
|
||||
resetDownloaderImpl(segment_lock);
|
||||
if (download_state == State::DOWNLOADING) /// != in case of completeWithState
|
||||
resetDownloadingStateUnlocked(segment_lock);
|
||||
resetDownloaderUnlocked(segment_lock);
|
||||
}
|
||||
|
||||
switch (download_state)
|
||||
@ -535,16 +563,17 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard<std::mutex> & cach
|
||||
}
|
||||
case State::DOWNLOADED:
|
||||
{
|
||||
assert(getDownloadedSizeUnlocked(segment_lock) == range().size());
|
||||
assert(isDownloadedUnlocked(segment_lock));
|
||||
chassert(getDownloadedSizeUnlocked(segment_lock) == range().size());
|
||||
assert(is_downloaded);
|
||||
assert(!cache_writer);
|
||||
break;
|
||||
}
|
||||
case State::DOWNLOADING:
|
||||
case State::EMPTY:
|
||||
{
|
||||
assert(!is_last_holder);
|
||||
chassert(!is_last_holder);
|
||||
break;
|
||||
}
|
||||
case State::EMPTY:
|
||||
case State::PARTIALLY_DOWNLOADED:
|
||||
case State::PARTIALLY_DOWNLOADED_NO_CONTINUATION:
|
||||
{
|
||||
@ -554,7 +583,7 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard<std::mutex> & cach
|
||||
{
|
||||
LOG_TEST(log, "Remove cell {} (nothing downloaded)", range().toString());
|
||||
|
||||
download_state = State::SKIP_CACHE;
|
||||
setDownloadState(State::SKIP_CACHE);
|
||||
cache->remove(key(), offset(), cache_lock, segment_lock);
|
||||
}
|
||||
else
|
||||
@ -567,7 +596,7 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard<std::mutex> & cach
|
||||
* in FileSegmentsHolder represent a contiguous range, so we can resize
|
||||
* it only when nobody needs it.
|
||||
*/
|
||||
download_state = State::PARTIALLY_DOWNLOADED_NO_CONTINUATION;
|
||||
setDownloadState(State::PARTIALLY_DOWNLOADED_NO_CONTINUATION);
|
||||
|
||||
/// Resize this file segment by creating a copy file segment with DOWNLOADED state,
|
||||
/// but current file segment should remain PARRTIALLY_DOWNLOADED_NO_CONTINUATION and with detached state,
|
||||
@ -576,23 +605,22 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard<std::mutex> & cach
|
||||
cache->reduceSizeToDownloaded(key(), offset(), cache_lock, segment_lock);
|
||||
}
|
||||
|
||||
markAsDetached(segment_lock);
|
||||
detachAssumeStateFinalized(segment_lock);
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
LOG_TEST(log, "Completed file segment: {}", getInfoForLogImpl(segment_lock));
|
||||
assertCorrectnessImpl(segment_lock);
|
||||
LOG_TEST(log, "Completed file segment: {}", getInfoForLogUnlocked(segment_lock));
|
||||
}
|
||||
|
||||
String FileSegment::getInfoForLog() const
|
||||
{
|
||||
std::lock_guard segment_lock(mutex);
|
||||
return getInfoForLogImpl(segment_lock);
|
||||
std::unique_lock segment_lock(mutex);
|
||||
return getInfoForLogUnlocked(segment_lock);
|
||||
}
|
||||
|
||||
String FileSegment::getInfoForLogImpl(std::lock_guard<std::mutex> & segment_lock) const
|
||||
String FileSegment::getInfoForLogUnlocked(std::unique_lock<std::mutex> & segment_lock) const
|
||||
{
|
||||
WriteBufferFromOwnString info;
|
||||
info << "File segment: " << range().toString() << ", ";
|
||||
@ -601,15 +629,18 @@ String FileSegment::getInfoForLogImpl(std::lock_guard<std::mutex> & segment_lock
|
||||
info << "downloaded size: " << getDownloadedSizeUnlocked(segment_lock) << ", ";
|
||||
info << "reserved size: " << reserved_size << ", ";
|
||||
info << "downloader id: " << (downloader_id.empty() ? "None" : downloader_id) << ", ";
|
||||
info << "current write offset: " << getCurrentWriteOffsetUnlocked(segment_lock) << ", ";
|
||||
info << "first non-downloaded offset: " << getFirstNonDownloadedOffsetUnlocked(segment_lock) << ", ";
|
||||
info << "caller id: " << getCallerId() << ", ";
|
||||
info << "detached: " << is_detached << ", ";
|
||||
info << "persistent: " << is_persistent;
|
||||
|
||||
return info.str();
|
||||
}
|
||||
|
||||
void FileSegment::wrapWithCacheInfo(Exception & e, const String & message, std::lock_guard<std::mutex> & segment_lock) const
|
||||
void FileSegment::wrapWithCacheInfo(Exception & e, const String & message, std::unique_lock<std::mutex> & segment_lock) const
|
||||
{
|
||||
e.addMessage(fmt::format("{}, current cache state: {}", message, getInfoForLogImpl(segment_lock)));
|
||||
e.addMessage(fmt::format("{}, current cache state: {}", message, getInfoForLogUnlocked(segment_lock)));
|
||||
}
|
||||
|
||||
String FileSegment::stateToString(FileSegment::State state)
|
||||
@ -634,63 +665,64 @@ String FileSegment::stateToString(FileSegment::State state)
|
||||
|
||||
void FileSegment::assertCorrectness() const
|
||||
{
|
||||
std::lock_guard segment_lock(mutex);
|
||||
assertCorrectnessImpl(segment_lock);
|
||||
std::unique_lock segment_lock(mutex);
|
||||
assertCorrectnessUnlocked(segment_lock);
|
||||
}
|
||||
|
||||
void FileSegment::assertCorrectnessImpl(std::lock_guard<std::mutex> & /* segment_lock */) const
|
||||
void FileSegment::assertCorrectnessUnlocked(std::unique_lock<std::mutex> & segment_lock) const
|
||||
{
|
||||
assert(downloader_id.empty() == (download_state != FileSegment::State::DOWNLOADING));
|
||||
assert(!downloader_id.empty() == (download_state == FileSegment::State::DOWNLOADING));
|
||||
assert(download_state != FileSegment::State::DOWNLOADED || std::filesystem::file_size(getPathInLocalCache()) > 0);
|
||||
auto current_downloader = getDownloaderUnlocked(segment_lock);
|
||||
chassert(current_downloader.empty() == (download_state != FileSegment::State::DOWNLOADING));
|
||||
chassert(!current_downloader.empty() == (download_state == FileSegment::State::DOWNLOADING));
|
||||
chassert(download_state != FileSegment::State::DOWNLOADED || std::filesystem::file_size(getPathInLocalCache()) > 0);
|
||||
}
|
||||
|
||||
void FileSegment::throwIfDetached() const
|
||||
{
|
||||
std::lock_guard segment_lock(mutex);
|
||||
throwIfDetachedUnlocked(segment_lock);
|
||||
}
|
||||
|
||||
void FileSegment::throwIfDetachedUnlocked(std::lock_guard<std::mutex> & segment_lock) const
|
||||
void FileSegment::throwIfDetachedUnlocked(std::unique_lock<std::mutex> & segment_lock) const
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Cache file segment is in detached state, operation not allowed. "
|
||||
"It can happen when cache was concurrently dropped with SYSTEM DROP FILESYSTEM CACHE FORCE. "
|
||||
"Please, retry. File segment info: {}", getInfoForLogImpl(segment_lock));
|
||||
"Please, retry. File segment info: {}", getInfoForLogUnlocked(segment_lock));
|
||||
}
|
||||
|
||||
void FileSegment::assertNotDetached() const
|
||||
{
|
||||
std::unique_lock segment_lock(mutex);
|
||||
assertNotDetachedUnlocked(segment_lock);
|
||||
}
|
||||
|
||||
void FileSegment::assertNotDetached(std::lock_guard<std::mutex> & segment_lock) const
|
||||
void FileSegment::assertNotDetachedUnlocked(std::unique_lock<std::mutex> & segment_lock) const
|
||||
{
|
||||
if (is_detached)
|
||||
throwIfDetachedUnlocked(segment_lock);
|
||||
}
|
||||
|
||||
void FileSegment::assertDetachedStatus(std::lock_guard<std::mutex> & segment_lock) const
|
||||
void FileSegment::assertDetachedStatus(std::unique_lock<std::mutex> & segment_lock) const
|
||||
{
|
||||
/// Detached file segment is allowed to have only a certain subset of states.
|
||||
/// It should be either EMPTY or one of the finalized states.
|
||||
|
||||
if (download_state != State::EMPTY && !hasFinalizedState())
|
||||
if (download_state != State::EMPTY && !hasFinalizedStateUnlocked(segment_lock))
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Detached file segment has incorrect state: {}",
|
||||
getInfoForLogImpl(segment_lock));
|
||||
getInfoForLogUnlocked(segment_lock));
|
||||
}
|
||||
}
|
||||
|
||||
FileSegmentPtr FileSegment::getSnapshot(const FileSegmentPtr & file_segment, std::lock_guard<std::mutex> & /* cache_lock */)
|
||||
{
|
||||
std::lock_guard segment_lock(file_segment->mutex);
|
||||
std::unique_lock segment_lock(file_segment->mutex);
|
||||
|
||||
auto snapshot = std::make_shared<FileSegment>(
|
||||
file_segment->offset(),
|
||||
file_segment->range().size(),
|
||||
file_segment->key(),
|
||||
nullptr,
|
||||
State::EMPTY);
|
||||
State::EMPTY,
|
||||
CreateFileSegmentSettings{});
|
||||
|
||||
snapshot->hits_count = file_segment->getHitsCount();
|
||||
snapshot->ref_count = file_segment.use_count();
|
||||
@ -701,41 +733,43 @@ FileSegmentPtr FileSegment::getSnapshot(const FileSegmentPtr & file_segment, std
|
||||
return snapshot;
|
||||
}
|
||||
|
||||
bool FileSegment::hasFinalizedState() const
|
||||
bool FileSegment::hasFinalizedStateUnlocked(std::unique_lock<std::mutex> & /* segment_lock */) const
|
||||
{
|
||||
return download_state == State::DOWNLOADED
|
||||
|| download_state == State::PARTIALLY_DOWNLOADED_NO_CONTINUATION
|
||||
|| download_state == State::SKIP_CACHE;
|
||||
}
|
||||
|
||||
void FileSegment::detach(
|
||||
std::lock_guard<std::mutex> & /* cache_lock */,
|
||||
std::lock_guard<std::mutex> & segment_lock)
|
||||
bool FileSegment::isDetached() const
|
||||
{
|
||||
std::unique_lock segment_lock(mutex);
|
||||
return is_detached;
|
||||
}
|
||||
|
||||
void FileSegment::detach(std::lock_guard<std::mutex> & /* cache_lock */, std::unique_lock<std::mutex> & segment_lock)
|
||||
{
|
||||
/// Now detached status can be in 2 cases, which do not do any complex logic:
|
||||
/// 1. there is only 1 remaining file segment holder
|
||||
/// && it does not need this segment anymore
|
||||
/// && this file segment was in cache and needs to be removed
|
||||
/// 2. in read_from_cache_if_exists_otherwise_bypass_cache case
|
||||
if (is_detached)
|
||||
return;
|
||||
|
||||
markAsDetached(segment_lock);
|
||||
download_state = State::PARTIALLY_DOWNLOADED_NO_CONTINUATION;
|
||||
downloader_id.clear();
|
||||
if (download_state == State::DOWNLOADING)
|
||||
resetDownloadingStateUnlocked(segment_lock);
|
||||
else
|
||||
setDownloadState(State::PARTIALLY_DOWNLOADED_NO_CONTINUATION);
|
||||
|
||||
LOG_DEBUG(log, "Detached file segment: {}", getInfoForLogImpl(segment_lock));
|
||||
resetDownloaderUnlocked(segment_lock);
|
||||
detachAssumeStateFinalized(segment_lock);
|
||||
}
|
||||
|
||||
void FileSegment::markAsDetached(std::lock_guard<std::mutex> & /* segment_lock */)
|
||||
void FileSegment::detachAssumeStateFinalized(std::unique_lock<std::mutex> & segment_lock)
|
||||
{
|
||||
is_detached = true;
|
||||
CurrentMetrics::add(CurrentMetrics::CacheDetachedFileSegments);
|
||||
LOG_TEST(log, "Detached file segment: {}", getInfoForLogUnlocked(segment_lock));
|
||||
}
|
||||
|
||||
FileSegment::~FileSegment()
|
||||
{
|
||||
std::lock_guard segment_lock(mutex);
|
||||
std::unique_lock segment_lock(mutex);
|
||||
if (is_detached)
|
||||
CurrentMetrics::sub(CurrentMetrics::CacheDetachedFileSegments);
|
||||
}
|
||||
@ -761,7 +795,7 @@ FileSegmentsHolder::~FileSegmentsHolder()
|
||||
bool is_detached = false;
|
||||
|
||||
{
|
||||
std::lock_guard segment_lock(file_segment->mutex);
|
||||
std::unique_lock segment_lock(file_segment->mutex);
|
||||
is_detached = file_segment->isDetached(segment_lock);
|
||||
if (is_detached)
|
||||
file_segment->assertDetachedStatus(segment_lock);
|
||||
@ -779,7 +813,7 @@ FileSegmentsHolder::~FileSegmentsHolder()
|
||||
/// under the same mutex, because complete() checks for segment pointers.
|
||||
std::lock_guard cache_lock(cache->mutex);
|
||||
|
||||
file_segment->completeWithoutState(cache_lock);
|
||||
file_segment->completeWithoutStateUnlocked(cache_lock);
|
||||
|
||||
file_segment_it = file_segments.erase(current_file_segment_it);
|
||||
}
|
||||
@ -822,10 +856,16 @@ FileSegments::iterator FileSegmentRangeWriter::allocateFileSegment(size_t offset
|
||||
|
||||
std::lock_guard cache_lock(cache->mutex);
|
||||
|
||||
CreateFileSegmentSettings create_settings
|
||||
{
|
||||
.is_persistent = is_persistent,
|
||||
};
|
||||
|
||||
/// We set max_file_segment_size to be downloaded,
|
||||
/// if we have less size to write, file segment will be resized in complete() method.
|
||||
auto file_segment = cache->createFileSegmentForDownload(
|
||||
key, offset, cache->max_file_segment_size, is_persistent, cache_lock);
|
||||
key, offset, cache->max_file_segment_size, create_settings, cache_lock);
|
||||
|
||||
return file_segments_holder.add(std::move(file_segment));
|
||||
}
|
||||
|
||||
@ -850,19 +890,18 @@ void FileSegmentRangeWriter::completeFileSegment(FileSegment & file_segment)
|
||||
/// and therefore cannot be concurrently accessed. Nevertheless, it can be
|
||||
/// accessed by cache system tables if someone read from them,
|
||||
/// therefore we need a mutex.
|
||||
std::lock_guard segment_lock(file_segment.mutex);
|
||||
std::unique_lock segment_lock(file_segment.mutex);
|
||||
|
||||
assert(current_downloaded_size <= file_segment.range().size());
|
||||
file_segment.segment_range = FileSegment::Range(
|
||||
file_segment.segment_range.left,
|
||||
file_segment.segment_range.left + current_downloaded_size - 1);
|
||||
file_segment.reserved_size = current_downloaded_size;
|
||||
|
||||
file_segment.setDownloadedUnlocked(segment_lock);
|
||||
}
|
||||
|
||||
{
|
||||
std::lock_guard cache_lock(cache->mutex);
|
||||
file_segment.completeWithoutState(cache_lock);
|
||||
}
|
||||
file_segment.completeWithoutState();
|
||||
|
||||
on_complete_file_segment_func(file_segment);
|
||||
}
|
||||
@ -893,18 +932,20 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset
|
||||
offset, current_file_segment_write_offset);
|
||||
}
|
||||
|
||||
size_t current_write_offset = (*current_file_segment_it)->getCurrentWriteOffset();
|
||||
|
||||
auto current_file_segment = *current_file_segment_it;
|
||||
if (current_file_segment->getRemainingSizeToDownload() == 0)
|
||||
{
|
||||
completeFileSegment(*current_file_segment);
|
||||
current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, is_persistent);
|
||||
}
|
||||
else if (current_file_segment->getDownloadOffset() != offset)
|
||||
else if (current_write_offset != offset)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Cannot file segment download offset {} does not match current write offset {}",
|
||||
current_file_segment->getDownloadOffset(), offset);
|
||||
current_write_offset, offset);
|
||||
}
|
||||
}
|
||||
|
||||
@ -915,7 +956,10 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to set a downloader. ({})", file_segment->getInfoForLog());
|
||||
|
||||
SCOPE_EXIT({
|
||||
file_segment->resetDownloader();
|
||||
if (file_segment->isDownloader())
|
||||
{
|
||||
file_segment->completePartAndResetDownloader();
|
||||
}
|
||||
});
|
||||
|
||||
bool reserved = file_segment->reserve(size);
|
||||
@ -932,7 +976,17 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset
|
||||
return false;
|
||||
}
|
||||
|
||||
(*current_file_segment_it)->write(data, size, offset);
|
||||
try
|
||||
{
|
||||
file_segment->write(data, size, offset);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
file_segment->completePartAndResetDownloader();
|
||||
throw;
|
||||
}
|
||||
|
||||
file_segment->completePartAndResetDownloader();
|
||||
current_file_segment_write_offset += size;
|
||||
|
||||
return true;
|
||||
|
@ -1,11 +1,15 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <Interpreters/Cache/FileCacheKey.h>
|
||||
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/ReadBufferFromFileBase.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <base/getThreadId.h>
|
||||
#include <list>
|
||||
#include <Interpreters/Cache/FileCacheKey.h>
|
||||
#include <queue>
|
||||
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
@ -26,17 +30,25 @@ using FileSegmentPtr = std::shared_ptr<FileSegment>;
|
||||
using FileSegments = std::list<FileSegmentPtr>;
|
||||
|
||||
|
||||
class FileSegment : boost::noncopyable
|
||||
struct CreateFileSegmentSettings
|
||||
{
|
||||
bool is_persistent = false;
|
||||
};
|
||||
|
||||
class FileSegment : private boost::noncopyable, public std::enable_shared_from_this<FileSegment>
|
||||
{
|
||||
|
||||
friend class FileCache;
|
||||
friend struct FileSegmentsHolder;
|
||||
friend class FileSegmentRangeWriter;
|
||||
friend class StorageSystemFilesystemCache;
|
||||
|
||||
public:
|
||||
using Key = FileCacheKey;
|
||||
using RemoteFileReaderPtr = std::shared_ptr<ReadBufferFromFileBase>;
|
||||
using LocalCacheWriterPtr = std::unique_ptr<WriteBufferFromFile>;
|
||||
using Downloader = std::string;
|
||||
using DownloaderId = std::string;
|
||||
|
||||
enum class State
|
||||
{
|
||||
@ -78,7 +90,7 @@ public:
|
||||
const Key & key_,
|
||||
FileCache * cache_,
|
||||
State download_state_,
|
||||
bool is_persistent_ = false);
|
||||
const CreateFileSegmentSettings & create_settings);
|
||||
|
||||
~FileSegment();
|
||||
|
||||
@ -101,6 +113,14 @@ public:
|
||||
String toString() const { return fmt::format("[{}, {}]", std::to_string(left), std::to_string(right)); }
|
||||
};
|
||||
|
||||
static String getCallerId();
|
||||
|
||||
String getInfoForLog() const;
|
||||
|
||||
/**
|
||||
* ========== Methods to get file segment's constant state ==================
|
||||
*/
|
||||
|
||||
const Range & range() const { return segment_range; }
|
||||
|
||||
const Key & key() const { return file_key; }
|
||||
@ -109,11 +129,85 @@ public:
|
||||
|
||||
bool isPersistent() const { return is_persistent; }
|
||||
|
||||
using UniqueId = std::pair<FileCacheKey, size_t>;
|
||||
UniqueId getUniqueId() const { return std::pair(key(), offset()); }
|
||||
|
||||
String getPathInLocalCache() const;
|
||||
|
||||
/**
|
||||
* ========== Methods for _any_ file segment's owner ========================
|
||||
*/
|
||||
|
||||
String getOrSetDownloader();
|
||||
|
||||
bool isDownloader() const;
|
||||
|
||||
DownloaderId getDownloader() const;
|
||||
|
||||
/// Wait for the change of state from DOWNLOADING to any other.
|
||||
State wait();
|
||||
|
||||
bool reserve(size_t size);
|
||||
bool isDownloaded() const;
|
||||
|
||||
void write(const char * from, size_t size, size_t offset_);
|
||||
size_t getHitsCount() const { return hits_count; }
|
||||
|
||||
size_t getRefCount() const { return ref_count; }
|
||||
|
||||
void incrementHitsCount() { ++hits_count; }
|
||||
|
||||
size_t getCurrentWriteOffset() const;
|
||||
|
||||
size_t getFirstNonDownloadedOffset() const;
|
||||
|
||||
size_t getDownloadedSize() const;
|
||||
|
||||
/// Now detached status can be used in the following cases:
|
||||
/// 1. there is only 1 remaining file segment holder
|
||||
/// && it does not need this segment anymore
|
||||
/// && this file segment was in cache and needs to be removed
|
||||
/// 2. in read_from_cache_if_exists_otherwise_bypass_cache case to create NOOP file segments.
|
||||
/// 3. removeIfExists - method which removes file segments from cache even though
|
||||
/// it might be used at the moment.
|
||||
|
||||
/// If file segment is detached it means the following:
|
||||
/// 1. It is not present in FileCache, e.g. will not be visible to any cache user apart from
|
||||
/// those who acquired shared pointer to this file segment before it was detached.
|
||||
/// 2. Detached file segment can still be hold by some cache users, but it's state became
|
||||
/// immutable at the point it was detached, any non-const / stateful method will throw an
|
||||
/// exception.
|
||||
void detach(std::lock_guard<std::mutex> & cache_lock, std::unique_lock<std::mutex> & segment_lock);
|
||||
|
||||
static FileSegmentPtr getSnapshot(const FileSegmentPtr & file_segment, std::lock_guard<std::mutex> & cache_lock);
|
||||
|
||||
bool isDetached() const;
|
||||
|
||||
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.
|
||||
bool reserve(size_t size_to_reserve);
|
||||
|
||||
/// Write data into reserved space.
|
||||
void write(const char * from, size_t size, size_t offset);
|
||||
|
||||
/// Complete file segment with a certain state.
|
||||
void completeWithState(State state);
|
||||
|
||||
void completeWithoutState();
|
||||
|
||||
/// Complete file segment's part which was last written.
|
||||
void completePartAndResetDownloader();
|
||||
|
||||
void resetDownloader();
|
||||
|
||||
RemoteFileReaderPtr getRemoteFileReader();
|
||||
|
||||
@ -123,91 +217,55 @@ public:
|
||||
|
||||
void resetRemoteFileReader();
|
||||
|
||||
String getOrSetDownloader();
|
||||
|
||||
String getDownloader() const;
|
||||
|
||||
void resetDownloader();
|
||||
|
||||
bool isDownloader() const;
|
||||
|
||||
bool isDownloaded() const;
|
||||
|
||||
static String getCallerId();
|
||||
|
||||
size_t getDownloadOffset() const;
|
||||
|
||||
size_t getDownloadedSize() const;
|
||||
|
||||
size_t getRemainingSizeToDownload() const;
|
||||
|
||||
void completeBatchAndResetDownloader();
|
||||
|
||||
void completeWithState(State state);
|
||||
|
||||
String getInfoForLog() const;
|
||||
|
||||
size_t getHitsCount() const { return hits_count; }
|
||||
|
||||
size_t getRefCount() const { return ref_count; }
|
||||
|
||||
void incrementHitsCount() { ++hits_count; }
|
||||
|
||||
void assertCorrectness() const;
|
||||
|
||||
static FileSegmentPtr getSnapshot(
|
||||
const FileSegmentPtr & file_segment,
|
||||
std::lock_guard<std::mutex> & cache_lock);
|
||||
|
||||
void detach(
|
||||
std::lock_guard<std::mutex> & cache_lock,
|
||||
std::lock_guard<std::mutex> & segment_lock);
|
||||
|
||||
[[noreturn]] void throwIfDetached() const;
|
||||
|
||||
bool isDetached() const;
|
||||
|
||||
String getPathInLocalCache() const;
|
||||
|
||||
private:
|
||||
size_t availableSize() const { return reserved_size - downloaded_size; }
|
||||
size_t getFirstNonDownloadedOffsetUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
|
||||
size_t getCurrentWriteOffsetUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
|
||||
size_t getDownloadedSizeUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
|
||||
|
||||
size_t getDownloadedSizeUnlocked(std::lock_guard<std::mutex> & segment_lock) const;
|
||||
String getInfoForLogImpl(std::lock_guard<std::mutex> & segment_lock) const;
|
||||
void assertCorrectnessImpl(std::lock_guard<std::mutex> & segment_lock) const;
|
||||
bool hasFinalizedState() const;
|
||||
String getInfoForLogUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
|
||||
|
||||
bool isDetached(std::lock_guard<std::mutex> & /* segment_lock */) const { return is_detached; }
|
||||
void markAsDetached(std::lock_guard<std::mutex> & segment_lock);
|
||||
[[noreturn]] void throwIfDetachedUnlocked(std::lock_guard<std::mutex> & segment_lock) const;
|
||||
String getDownloaderUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
|
||||
void resetDownloaderUnlocked(std::unique_lock<std::mutex> & segment_lock);
|
||||
void resetDownloadingStateUnlocked(std::unique_lock<std::mutex> & segment_lock);
|
||||
|
||||
void assertDetachedStatus(std::lock_guard<std::mutex> & segment_lock) const;
|
||||
void assertNotDetached(std::lock_guard<std::mutex> & segment_lock) const;
|
||||
void setDownloadState(State state);
|
||||
|
||||
void setDownloaded(std::lock_guard<std::mutex> & segment_lock);
|
||||
void setDownloadFailed(std::lock_guard<std::mutex> & segment_lock);
|
||||
bool isDownloaderImpl(std::lock_guard<std::mutex> & segment_lock) const;
|
||||
void setDownloadedUnlocked(std::unique_lock<std::mutex> & segment_lock);
|
||||
void setDownloadFailedUnlocked(std::unique_lock<std::mutex> & segment_lock);
|
||||
|
||||
bool isDownloadedUnlocked(std::lock_guard<std::mutex> & segment_lock) const;
|
||||
bool hasFinalizedStateUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
|
||||
|
||||
void wrapWithCacheInfo(Exception & e, const String & message, std::lock_guard<std::mutex> & segment_lock) const;
|
||||
bool isDownloaderUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
|
||||
|
||||
bool lastFileSegmentHolder() const;
|
||||
bool isDetached(std::unique_lock<std::mutex> & /* segment_lock */) const { return is_detached; }
|
||||
void detachAssumeStateFinalized(std::unique_lock<std::mutex> & segment_lock);
|
||||
[[noreturn]] void throwIfDetachedUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
|
||||
|
||||
void assertDetachedStatus(std::unique_lock<std::mutex> & segment_lock) const;
|
||||
void assertNotDetached() const;
|
||||
void assertNotDetachedUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
|
||||
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
|
||||
/// of the file segment pointer must be done under the same cache mutex.
|
||||
void completeBasedOnCurrentState(std::lock_guard<std::mutex> & cache_lock, std::lock_guard<std::mutex> & segment_lock);
|
||||
void completeWithoutState(std::lock_guard<std::mutex> & cache_lock);
|
||||
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);
|
||||
|
||||
void resetDownloaderImpl(std::lock_guard<std::mutex> & segment_lock);
|
||||
void completePartAndResetDownloaderUnlocked(std::unique_lock<std::mutex> & segment_lock);
|
||||
|
||||
void wrapWithCacheInfo(Exception & e, const String & message, std::unique_lock<std::mutex> & segment_lock) const;
|
||||
|
||||
Range segment_range;
|
||||
|
||||
State download_state;
|
||||
|
||||
String downloader_id;
|
||||
/// The one who prepares the download
|
||||
DownloaderId downloader_id;
|
||||
|
||||
RemoteFileReaderPtr remote_file_reader;
|
||||
LocalCacheWriterPtr cache_writer;
|
||||
@ -245,6 +303,7 @@ private:
|
||||
std::atomic<size_t> ref_count = 0; /// Used for getting snapshot state
|
||||
|
||||
bool is_persistent;
|
||||
|
||||
CurrentMetrics::Increment metric_increment{CurrentMetrics::CacheFileSegments};
|
||||
};
|
||||
|
||||
|
@ -143,7 +143,7 @@ namespace ErrorCodes
|
||||
/** Set of known objects (environment), that could be used in query.
|
||||
* Shared (global) part. Order of members (especially, order of destruction) is very important.
|
||||
*/
|
||||
struct ContextSharedPart
|
||||
struct ContextSharedPart : boost::noncopyable
|
||||
{
|
||||
Poco::Logger * log = &Poco::Logger::get("Context");
|
||||
|
||||
@ -215,6 +215,7 @@ struct ContextSharedPart
|
||||
std::unique_ptr<AccessControl> access_control;
|
||||
mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks.
|
||||
mutable MarkCachePtr mark_cache; /// Cache of marks in compressed files.
|
||||
mutable std::unique_ptr<ThreadPool> load_marks_threadpool; /// Threadpool for loading marks cache.
|
||||
mutable UncompressedCachePtr index_uncompressed_cache; /// The cache of decompressed blocks for MergeTree indices.
|
||||
mutable MarkCachePtr index_mark_cache; /// Cache of marks in compressed files of MergeTree indices.
|
||||
mutable MMappedFileCachePtr mmap_cache; /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads.
|
||||
@ -313,11 +314,19 @@ struct ContextSharedPart
|
||||
|
||||
~ContextSharedPart()
|
||||
{
|
||||
/// Wait for thread pool for background writes,
|
||||
/// since it may use per-user MemoryTracker which will be destroyed here.
|
||||
try
|
||||
{
|
||||
/// Wait for thread pool for background writes,
|
||||
/// since it may use per-user MemoryTracker which will be destroyed here.
|
||||
IObjectStorage::getThreadPoolWriter().wait();
|
||||
/// Make sure that threadpool is destructed before this->process_list
|
||||
/// because thread_status, which was created for threads inside threadpool,
|
||||
/// relies on it.
|
||||
if (load_marks_threadpool)
|
||||
{
|
||||
load_marks_threadpool->wait();
|
||||
load_marks_threadpool.reset();
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -1688,6 +1697,17 @@ void Context::dropMarkCache() const
|
||||
shared->mark_cache->reset();
|
||||
}
|
||||
|
||||
ThreadPool & Context::getLoadMarksThreadpool() const
|
||||
{
|
||||
auto lock = getLock();
|
||||
if (!shared->load_marks_threadpool)
|
||||
{
|
||||
constexpr size_t pool_size = 50;
|
||||
constexpr size_t queue_size = 1000000;
|
||||
shared->load_marks_threadpool = std::make_unique<ThreadPool>(pool_size, pool_size, queue_size);
|
||||
}
|
||||
return *shared->load_marks_threadpool;
|
||||
}
|
||||
|
||||
void Context::setIndexUncompressedCache(size_t max_size_in_bytes)
|
||||
{
|
||||
@ -2974,7 +2994,7 @@ const IHostContextPtr & Context::getHostContext() const
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<ActionLocksManager> Context::getActionLocksManager()
|
||||
std::shared_ptr<ActionLocksManager> Context::getActionLocksManager() const
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
@ -3429,6 +3449,8 @@ ReadSettings Context::getReadSettings() const
|
||||
res.local_fs_prefetch = settings.local_filesystem_read_prefetch;
|
||||
res.remote_fs_prefetch = settings.remote_filesystem_read_prefetch;
|
||||
|
||||
res.load_marks_asynchronously = settings.load_marks_asynchronously;
|
||||
|
||||
res.remote_fs_read_max_backoff_ms = settings.remote_fs_read_max_backoff_ms;
|
||||
res.remote_fs_read_backoff_max_tries = settings.remote_fs_read_backoff_max_tries;
|
||||
res.enable_filesystem_cache = settings.enable_filesystem_cache;
|
||||
|
@ -806,6 +806,7 @@ public:
|
||||
void setMarkCache(size_t cache_size_in_bytes, const String & mark_cache_policy);
|
||||
std::shared_ptr<MarkCache> getMarkCache() const;
|
||||
void dropMarkCache() const;
|
||||
ThreadPool & getLoadMarksThreadpool() const;
|
||||
|
||||
/// Create a cache of index uncompressed blocks of specified size. This can be done only once.
|
||||
void setIndexUncompressedCache(size_t max_size_in_bytes);
|
||||
@ -936,7 +937,7 @@ public:
|
||||
bool applyDeletedMask() const { return apply_deleted_mask; }
|
||||
void setApplyDeletedMask(bool apply) { apply_deleted_mask = apply; }
|
||||
|
||||
ActionLocksManagerPtr getActionLocksManager();
|
||||
ActionLocksManagerPtr getActionLocksManager() const;
|
||||
|
||||
enum class ApplicationType
|
||||
{
|
||||
|
@ -890,7 +890,7 @@ void DDLWorker::cleanupQueue(Int64, const ZooKeeperPtr & zookeeper)
|
||||
|
||||
/// We recursively delete all nodes except node_path/finished to prevent staled hosts from
|
||||
/// creating node_path/active node (see createStatusDirs(...))
|
||||
zookeeper->tryRemoveChildrenRecursive(node_path, /* probably_flat */ false, "finished");
|
||||
zookeeper->tryRemoveChildrenRecursive(node_path, /* probably_flat */ false, zkutil::RemoveException{"finished"});
|
||||
|
||||
/// And then we remove node_path and node_path/finished in a single transaction
|
||||
Coordination::Requests ops;
|
||||
|
@ -21,7 +21,7 @@ static String typeToString(FilesystemCacheLogElement::CacheType type)
|
||||
case FilesystemCacheLogElement::CacheType::READ_FROM_FS_BYPASSING_CACHE:
|
||||
return "READ_FROM_FS_BYPASSING_CACHE";
|
||||
case FilesystemCacheLogElement::CacheType::WRITE_THROUGH_CACHE:
|
||||
return "READ_FROM_FS_BYPASSING_CACHE";
|
||||
return "WRITE_THROUGH_CACHE";
|
||||
}
|
||||
__builtin_unreachable();
|
||||
}
|
||||
|
@ -180,30 +180,41 @@ void InterpreterSystemQuery::startStopAction(StorageActionBlockType action_type,
|
||||
{
|
||||
for (auto & elem : DatabaseCatalog::instance().getDatabases())
|
||||
{
|
||||
for (auto iterator = elem.second->getTablesIterator(getContext()); iterator->isValid(); iterator->next())
|
||||
{
|
||||
StoragePtr table = iterator->table();
|
||||
if (!table)
|
||||
continue;
|
||||
|
||||
if (!access->isGranted(required_access_type, elem.first, iterator->name()))
|
||||
{
|
||||
LOG_INFO(log, "Access {} denied, skipping {}.{}", toString(required_access_type), elem.first, iterator->name());
|
||||
continue;
|
||||
}
|
||||
|
||||
if (start)
|
||||
{
|
||||
manager->remove(table, action_type);
|
||||
table->onActionLockRemove(action_type);
|
||||
}
|
||||
else
|
||||
manager->add(table, action_type);
|
||||
}
|
||||
startStopActionInDatabase(action_type, start, elem.first, elem.second, getContext(), log);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void InterpreterSystemQuery::startStopActionInDatabase(StorageActionBlockType action_type, bool start,
|
||||
const String & database_name, const DatabasePtr & database,
|
||||
const ContextPtr & local_context, Poco::Logger * log)
|
||||
{
|
||||
auto manager = local_context->getActionLocksManager();
|
||||
auto access = local_context->getAccess();
|
||||
auto required_access_type = getRequiredAccessType(action_type);
|
||||
|
||||
for (auto iterator = database->getTablesIterator(local_context); iterator->isValid(); iterator->next())
|
||||
{
|
||||
StoragePtr table = iterator->table();
|
||||
if (!table)
|
||||
continue;
|
||||
|
||||
if (!access->isGranted(required_access_type, database_name, iterator->name()))
|
||||
{
|
||||
LOG_INFO(log, "Access {} denied, skipping {}.{}", toString(required_access_type), database_name, iterator->name());
|
||||
continue;
|
||||
}
|
||||
|
||||
if (start)
|
||||
{
|
||||
manager->remove(table, action_type);
|
||||
table->onActionLockRemove(action_type);
|
||||
}
|
||||
else
|
||||
manager->add(table, action_type);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
InterpreterSystemQuery::InterpreterSystemQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_)
|
||||
: WithMutableContext(context_), query_ptr(query_ptr_->clone()), log(&Poco::Logger::get("InterpreterSystemQuery"))
|
||||
@ -528,7 +539,7 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
{
|
||||
getContext()->checkAccess(AccessType::SYSTEM_UNFREEZE);
|
||||
/// The result contains information about deleted parts as a table. It is for compatibility with ALTER TABLE UNFREEZE query.
|
||||
result = Unfreezer().unfreeze(query.backup_name, getContext());
|
||||
result = Unfreezer(getContext()).systemUnfreeze(query.backup_name);
|
||||
break;
|
||||
}
|
||||
default:
|
||||
|
@ -16,6 +16,9 @@ namespace DB
|
||||
class Context;
|
||||
class AccessRightsElements;
|
||||
class ASTSystemQuery;
|
||||
class IDatabase;
|
||||
|
||||
using DatabasePtr = std::shared_ptr<IDatabase>;
|
||||
|
||||
|
||||
/** Implement various SYSTEM queries.
|
||||
@ -37,6 +40,10 @@ public:
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
static void startStopActionInDatabase(StorageActionBlockType action_type, bool start,
|
||||
const String & database_name, const DatabasePtr & database,
|
||||
const ContextPtr & local_context, Poco::Logger * log);
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
Poco::Logger * log = nullptr;
|
||||
|
@ -473,6 +473,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
|
||||
dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns);
|
||||
|
||||
std::vector<String> read_columns;
|
||||
/// First, break a sequence of commands into stages.
|
||||
for (auto & command : commands)
|
||||
{
|
||||
@ -706,17 +707,23 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
else if (command.type == MutationCommand::READ_COLUMN)
|
||||
{
|
||||
mutation_kind.set(MutationKind::MUTATE_OTHER);
|
||||
if (stages.empty() || !stages.back().column_to_updated.empty())
|
||||
stages.emplace_back(context);
|
||||
if (stages.size() == 1) /// First stage only supports filtering and can't update columns.
|
||||
stages.emplace_back(context);
|
||||
|
||||
stages.back().column_to_updated.emplace(command.column_name, std::make_shared<ASTIdentifier>(command.column_name));
|
||||
read_columns.emplace_back(command.column_name);
|
||||
}
|
||||
else
|
||||
throw Exception("Unknown mutation command type: " + DB::toString<int>(command.type), ErrorCodes::UNKNOWN_MUTATION_COMMAND);
|
||||
}
|
||||
|
||||
if (!read_columns.empty())
|
||||
{
|
||||
if (stages.empty() || !stages.back().column_to_updated.empty())
|
||||
stages.emplace_back(context);
|
||||
if (stages.size() == 1) /// First stage only supports filtering and can't update columns.
|
||||
stages.emplace_back(context);
|
||||
|
||||
for (auto & column_name : read_columns)
|
||||
stages.back().column_to_updated.emplace(column_name, std::make_shared<ASTIdentifier>(column_name));
|
||||
}
|
||||
|
||||
/// We care about affected indices and projections because we also need to rewrite them
|
||||
/// when one of index columns updated or filtered with delete.
|
||||
/// The same about columns, that are needed for calculation of TTL expressions.
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Parsers/parseQuery.h>
|
||||
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Interpreters/InterpreterSystemQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/loadMetadata.h>
|
||||
#include <Interpreters/executeQuery.h>
|
||||
@ -32,6 +33,14 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
namespace ActionLocks
|
||||
{
|
||||
extern StorageActionBlockType PartsMerge;
|
||||
extern StorageActionBlockType PartsFetch;
|
||||
extern StorageActionBlockType PartsSend;
|
||||
extern StorageActionBlockType DistributedSend;
|
||||
}
|
||||
|
||||
static void executeCreateQuery(
|
||||
const String & query,
|
||||
ContextMutablePtr context,
|
||||
@ -327,9 +336,24 @@ static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, cons
|
||||
}
|
||||
|
||||
auto local_context = Context::createCopy(context);
|
||||
|
||||
/// We have to stop background operations that may lock table for share to avoid DEADLOCK_AVOIDED error
|
||||
/// on moving tables from Ordinary database. Server has not started to accept connections yet,
|
||||
/// so there are no user queries, only background operations
|
||||
LOG_INFO(log, "Will stop background operations to be able to rename tables in Ordinary database {}", database_name);
|
||||
static const auto actions_to_stop = {
|
||||
ActionLocks::PartsMerge, ActionLocks::PartsFetch, ActionLocks::PartsSend, ActionLocks::DistributedSend
|
||||
};
|
||||
for (const auto & action : actions_to_stop)
|
||||
InterpreterSystemQuery::startStopActionInDatabase(action, /* start */ false, database_name, database, context, log);
|
||||
|
||||
local_context->setSetting("check_table_dependencies", false);
|
||||
convertOrdinaryDatabaseToAtomic(log, local_context, database, database_name, tmp_name);
|
||||
|
||||
LOG_INFO(log, "Will start background operations after renaming tables in database {}", database_name);
|
||||
for (const auto & action : actions_to_stop)
|
||||
InterpreterSystemQuery::startStopActionInDatabase(action, /* start */ true, database_name, database, context, log);
|
||||
|
||||
auto new_database = DatabaseCatalog::instance().getDatabase(database_name);
|
||||
UUID db_uuid = new_database->getUUID();
|
||||
std::vector<UUID> tables_uuids;
|
||||
|
@ -3,9 +3,9 @@
|
||||
#include <gtest/gtest.h>
|
||||
#include <Interpreters/Cache/FileCache.h>
|
||||
#include <Interpreters/Cache/FileSegment.h>
|
||||
#include <Interpreters/Cache/FileCacheSettings.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/filesystemHelpers.h>
|
||||
#include <Interpreters/Cache/FileCacheSettings.h>
|
||||
#include <Common/tests/gtest_global_context.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/hex.h>
|
||||
@ -64,7 +64,7 @@ void download(DB::FileSegmentPtr file_segment)
|
||||
fs::create_directories(subdir);
|
||||
|
||||
std::string data(size, '0');
|
||||
file_segment->write(data.data(), size, file_segment->getDownloadOffset());
|
||||
file_segment->write(data.data(), size, file_segment->getCurrentWriteOffset());
|
||||
}
|
||||
|
||||
void prepareAndDownload(DB::FileSegmentPtr file_segment)
|
||||
@ -89,6 +89,7 @@ TEST(FileCache, get)
|
||||
{
|
||||
if (fs::exists(cache_base_path))
|
||||
fs::remove_all(cache_base_path);
|
||||
fs::create_directories(cache_base_path);
|
||||
|
||||
DB::ThreadStatus thread_status;
|
||||
|
||||
@ -109,7 +110,7 @@ TEST(FileCache, get)
|
||||
auto key = cache.hash("key1");
|
||||
|
||||
{
|
||||
auto holder = cache.getOrSet(key, 0, 10, false); /// Add range [0, 9]
|
||||
auto holder = cache.getOrSet(key, 0, 10, {}); /// Add range [0, 9]
|
||||
auto segments = fromHolder(holder);
|
||||
/// Range was not present in cache. It should be added in cache as one while file segment.
|
||||
ASSERT_EQ(segments.size(), 1);
|
||||
@ -138,7 +139,7 @@ TEST(FileCache, get)
|
||||
|
||||
{
|
||||
/// Want range [5, 14], but [0, 9] already in cache, so only [10, 14] will be put in cache.
|
||||
auto holder = cache.getOrSet(key, 5, 10, false);
|
||||
auto holder = cache.getOrSet(key, 5, 10, {});
|
||||
auto segments = fromHolder(holder);
|
||||
ASSERT_EQ(segments.size(), 2);
|
||||
|
||||
@ -158,14 +159,14 @@ TEST(FileCache, get)
|
||||
ASSERT_EQ(cache.getUsedCacheSize(), 15);
|
||||
|
||||
{
|
||||
auto holder = cache.getOrSet(key, 9, 1, false); /// Get [9, 9]
|
||||
auto holder = cache.getOrSet(key, 9, 1, {}); /// Get [9, 9]
|
||||
auto segments = fromHolder(holder);
|
||||
ASSERT_EQ(segments.size(), 1);
|
||||
assertRange(7, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED);
|
||||
}
|
||||
|
||||
{
|
||||
auto holder = cache.getOrSet(key, 9, 2, false); /// Get [9, 10]
|
||||
auto holder = cache.getOrSet(key, 9, 2, {}); /// Get [9, 10]
|
||||
auto segments = fromHolder(holder);
|
||||
ASSERT_EQ(segments.size(), 2);
|
||||
assertRange(8, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED);
|
||||
@ -173,15 +174,15 @@ TEST(FileCache, get)
|
||||
}
|
||||
|
||||
{
|
||||
auto holder = cache.getOrSet(key, 10, 1, false); /// Get [10, 10]
|
||||
auto holder = cache.getOrSet(key, 10, 1, {}); /// Get [10, 10]
|
||||
auto segments = fromHolder(holder);
|
||||
ASSERT_EQ(segments.size(), 1);
|
||||
assertRange(10, segments[0], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED);
|
||||
}
|
||||
|
||||
complete(cache.getOrSet(key, 17, 4, false)); /// Get [17, 20]
|
||||
complete(cache.getOrSet(key, 24, 3, false)); /// Get [24, 26]
|
||||
/// complete(cache.getOrSet(key, 27, 1, false)); /// Get [27, 27]
|
||||
complete(cache.getOrSet(key, 17, 4, {})); /// Get [17, 20]
|
||||
complete(cache.getOrSet(key, 24, 3, {})); /// Get [24, 26]
|
||||
/// completeWithState(cache.getOrSet(key, 27, 1, false)); /// Get [27, 27]
|
||||
|
||||
/// Current cache: [__________][_____] [____] [___][]
|
||||
/// ^ ^^ ^ ^ ^ ^ ^^^
|
||||
@ -191,7 +192,7 @@ TEST(FileCache, get)
|
||||
ASSERT_EQ(cache.getUsedCacheSize(), 22);
|
||||
|
||||
{
|
||||
auto holder = cache.getOrSet(key, 0, 26, false); /// Get [0, 25]
|
||||
auto holder = cache.getOrSet(key, 0, 26, {}); /// Get [0, 25]
|
||||
auto segments = fromHolder(holder);
|
||||
ASSERT_EQ(segments.size(), 6);
|
||||
|
||||
@ -225,14 +226,14 @@ TEST(FileCache, get)
|
||||
/// as max elements size is reached, next attempt to put something in cache should fail.
|
||||
/// This will also check that [27, 27] was indeed evicted.
|
||||
|
||||
auto holder1 = cache.getOrSet(key, 27, 1, false);
|
||||
auto holder1 = cache.getOrSet(key, 27, 1, {});
|
||||
auto segments_1 = fromHolder(holder1); /// Get [27, 27]
|
||||
ASSERT_EQ(segments_1.size(), 1);
|
||||
assertRange(17, segments_1[0], DB::FileSegment::Range(27, 27), DB::FileSegment::State::EMPTY);
|
||||
}
|
||||
|
||||
{
|
||||
auto holder = cache.getOrSet(key, 12, 10, false); /// Get [12, 21]
|
||||
auto holder = cache.getOrSet(key, 12, 10, {}); /// Get [12, 21]
|
||||
auto segments = fromHolder(holder);
|
||||
ASSERT_EQ(segments.size(), 4);
|
||||
|
||||
@ -256,7 +257,7 @@ TEST(FileCache, get)
|
||||
ASSERT_EQ(cache.getFileSegmentsNum(), 5);
|
||||
|
||||
{
|
||||
auto holder = cache.getOrSet(key, 23, 5, false); /// Get [23, 28]
|
||||
auto holder = cache.getOrSet(key, 23, 5, {}); /// Get [23, 28]
|
||||
auto segments = fromHolder(holder);
|
||||
ASSERT_EQ(segments.size(), 3);
|
||||
|
||||
@ -277,12 +278,12 @@ TEST(FileCache, get)
|
||||
/// 17 21 2324 26 28
|
||||
|
||||
{
|
||||
auto holder5 = cache.getOrSet(key, 2, 3,false); /// Get [2, 4]
|
||||
auto holder5 = cache.getOrSet(key, 2, 3, {}); /// Get [2, 4]
|
||||
auto s5 = fromHolder(holder5);
|
||||
ASSERT_EQ(s5.size(), 1);
|
||||
assertRange(25, s5[0], DB::FileSegment::Range(2, 4), DB::FileSegment::State::EMPTY);
|
||||
|
||||
auto holder1 = cache.getOrSet(key, 30, 2, false); /// Get [30, 31]
|
||||
auto holder1 = cache.getOrSet(key, 30, 2, {}); /// Get [30, 31]
|
||||
auto s1 = fromHolder(holder1);
|
||||
ASSERT_EQ(s1.size(), 1);
|
||||
assertRange(26, s1[0], DB::FileSegment::Range(30, 31), DB::FileSegment::State::EMPTY);
|
||||
@ -298,20 +299,20 @@ TEST(FileCache, get)
|
||||
/// ^ ^ ^ ^ ^ ^ ^ ^
|
||||
/// 2 4 23 24 26 27 30 31
|
||||
|
||||
auto holder2 = cache.getOrSet(key, 23, 1, false); /// Get [23, 23]
|
||||
auto holder2 = cache.getOrSet(key, 23, 1, {}); /// Get [23, 23]
|
||||
auto s2 = fromHolder(holder2);
|
||||
ASSERT_EQ(s2.size(), 1);
|
||||
|
||||
auto holder3 = cache.getOrSet(key, 24, 3, false); /// Get [24, 26]
|
||||
auto holder3 = cache.getOrSet(key, 24, 3, {}); /// Get [24, 26]
|
||||
auto s3 = fromHolder(holder3);
|
||||
ASSERT_EQ(s3.size(), 1);
|
||||
|
||||
auto holder4 = cache.getOrSet(key, 27, 1, false); /// Get [27, 27]
|
||||
auto holder4 = cache.getOrSet(key, 27, 1, {}); /// Get [27, 27]
|
||||
auto s4 = fromHolder(holder4);
|
||||
ASSERT_EQ(s4.size(), 1);
|
||||
|
||||
/// All cache is now unreleasable because pointers are still hold
|
||||
auto holder6 = cache.getOrSet(key, 0, 40, false);
|
||||
auto holder6 = cache.getOrSet(key, 0, 40, {});
|
||||
auto f = fromHolder(holder6);
|
||||
ASSERT_EQ(f.size(), 9);
|
||||
|
||||
@ -332,7 +333,7 @@ TEST(FileCache, get)
|
||||
}
|
||||
|
||||
{
|
||||
auto holder = cache.getOrSet(key, 2, 3, false); /// Get [2, 4]
|
||||
auto holder = cache.getOrSet(key, 2, 3, {}); /// Get [2, 4]
|
||||
auto segments = fromHolder(holder);
|
||||
ASSERT_EQ(segments.size(), 1);
|
||||
assertRange(31, segments[0], DB::FileSegment::Range(2, 4), DB::FileSegment::State::DOWNLOADED);
|
||||
@ -343,7 +344,7 @@ TEST(FileCache, get)
|
||||
/// 2 4 23 24 26 27 30 31
|
||||
|
||||
{
|
||||
auto holder = cache.getOrSet(key, 25, 5, false); /// Get [25, 29]
|
||||
auto holder = cache.getOrSet(key, 25, 5, {}); /// Get [25, 29]
|
||||
auto segments = fromHolder(holder);
|
||||
ASSERT_EQ(segments.size(), 3);
|
||||
|
||||
@ -367,7 +368,7 @@ TEST(FileCache, get)
|
||||
DB::CurrentThread::QueryScope query_scope_holder_1(query_context_1);
|
||||
thread_status_1.attachQueryContext(query_context_1);
|
||||
|
||||
auto holder_2 = cache.getOrSet(key, 25, 5, false); /// Get [25, 29] once again.
|
||||
auto holder_2 = cache.getOrSet(key, 25, 5, {}); /// Get [25, 29] once again.
|
||||
auto segments_2 = fromHolder(holder_2);
|
||||
ASSERT_EQ(segments.size(), 3);
|
||||
|
||||
@ -406,11 +407,11 @@ TEST(FileCache, get)
|
||||
|
||||
{
|
||||
/// Now let's check the similar case but getting ERROR state after segment->wait(), when
|
||||
/// state is changed not manually via segment->complete(state) but from destructor of holder
|
||||
/// state is changed not manually via segment->completeWithState(state) but from destructor of holder
|
||||
/// and notify_all() is also called from destructor of holder.
|
||||
|
||||
std::optional<DB::FileSegmentsHolder> holder;
|
||||
holder.emplace(cache.getOrSet(key, 3, 23, false)); /// Get [3, 25]
|
||||
holder.emplace(cache.getOrSet(key, 3, 23, {})); /// Get [3, 25]
|
||||
|
||||
auto segments = fromHolder(*holder);
|
||||
ASSERT_EQ(segments.size(), 3);
|
||||
@ -436,7 +437,7 @@ TEST(FileCache, get)
|
||||
DB::CurrentThread::QueryScope query_scope_holder_1(query_context_1);
|
||||
thread_status_1.attachQueryContext(query_context_1);
|
||||
|
||||
auto holder_2 = cache.getOrSet(key, 3, 23, false); /// Get [3, 25] once again
|
||||
auto holder_2 = cache.getOrSet(key, 3, 23, {}); /// Get [3, 25] once again
|
||||
auto segments_2 = fromHolder(*holder);
|
||||
ASSERT_EQ(segments_2.size(), 3);
|
||||
|
||||
@ -485,7 +486,7 @@ TEST(FileCache, get)
|
||||
cache2.initialize();
|
||||
auto key = cache2.hash("key1");
|
||||
|
||||
auto holder1 = cache2.getOrSet(key, 2, 28, false); /// Get [2, 29]
|
||||
auto holder1 = cache2.getOrSet(key, 2, 28, {}); /// Get [2, 29]
|
||||
|
||||
auto segments1 = fromHolder(holder1);
|
||||
ASSERT_EQ(segments1.size(), 5);
|
||||
@ -506,7 +507,7 @@ TEST(FileCache, get)
|
||||
cache2.initialize();
|
||||
auto key = cache2.hash("key1");
|
||||
|
||||
auto holder1 = cache2.getOrSet(key, 0, 25, false); /// Get [0, 24]
|
||||
auto holder1 = cache2.getOrSet(key, 0, 25, {}); /// Get [0, 24]
|
||||
auto segments1 = fromHolder(holder1);
|
||||
|
||||
ASSERT_EQ(segments1.size(), 3);
|
||||
|
@ -201,6 +201,10 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &,
|
||||
if (!filesystem_cache_path.empty())
|
||||
settings.ostr << (settings.hilite ? hilite_none : "") << " " << filesystem_cache_path;
|
||||
}
|
||||
else if (type == Type::UNFREEZE)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(backup_name);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
@ -1158,18 +1158,20 @@ std::string PredefinedQueryHandler::getQuery(HTTPServerRequest & request, HTMLFo
|
||||
return predefined_query;
|
||||
}
|
||||
|
||||
HTTPRequestHandlerFactoryPtr createDynamicHandlerFactory(IServer & server, const std::string & config_prefix)
|
||||
HTTPRequestHandlerFactoryPtr createDynamicHandlerFactory(IServer & server,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix)
|
||||
{
|
||||
auto query_param_name = server.config().getString(config_prefix + ".handler.query_param_name", "query");
|
||||
auto query_param_name = config.getString(config_prefix + ".handler.query_param_name", "query");
|
||||
|
||||
std::optional<String> content_type_override;
|
||||
if (server.config().has(config_prefix + ".handler.content_type"))
|
||||
content_type_override = server.config().getString(config_prefix + ".handler.content_type");
|
||||
if (config.has(config_prefix + ".handler.content_type"))
|
||||
content_type_override = config.getString(config_prefix + ".handler.content_type");
|
||||
|
||||
auto factory = std::make_shared<HandlingRuleHTTPHandlerFactory<DynamicQueryHandler>>(
|
||||
server, std::move(query_param_name), std::move(content_type_override));
|
||||
|
||||
factory->addFiltersFromConfig(server.config(), config_prefix);
|
||||
factory->addFiltersFromConfig(config, config_prefix);
|
||||
|
||||
return factory;
|
||||
}
|
||||
@ -1197,23 +1199,23 @@ static inline CompiledRegexPtr getCompiledRegex(const std::string & expression)
|
||||
return compiled_regex;
|
||||
}
|
||||
|
||||
HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server, const std::string & config_prefix)
|
||||
HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration & configuration = server.config();
|
||||
|
||||
if (!configuration.has(config_prefix + ".handler.query"))
|
||||
if (!config.has(config_prefix + ".handler.query"))
|
||||
throw Exception("There is no path '" + config_prefix + ".handler.query' in configuration file.", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
|
||||
|
||||
std::string predefined_query = configuration.getString(config_prefix + ".handler.query");
|
||||
std::string predefined_query = config.getString(config_prefix + ".handler.query");
|
||||
NameSet analyze_receive_params = analyzeReceiveQueryParams(predefined_query);
|
||||
|
||||
std::unordered_map<String, CompiledRegexPtr> headers_name_with_regex;
|
||||
Poco::Util::AbstractConfiguration::Keys headers_name;
|
||||
configuration.keys(config_prefix + ".headers", headers_name);
|
||||
config.keys(config_prefix + ".headers", headers_name);
|
||||
|
||||
for (const auto & header_name : headers_name)
|
||||
{
|
||||
auto expression = configuration.getString(config_prefix + ".headers." + header_name);
|
||||
auto expression = config.getString(config_prefix + ".headers." + header_name);
|
||||
|
||||
if (!startsWith(expression, "regex:"))
|
||||
continue;
|
||||
@ -1225,14 +1227,14 @@ HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server, co
|
||||
}
|
||||
|
||||
std::optional<String> content_type_override;
|
||||
if (configuration.has(config_prefix + ".handler.content_type"))
|
||||
content_type_override = configuration.getString(config_prefix + ".handler.content_type");
|
||||
if (config.has(config_prefix + ".handler.content_type"))
|
||||
content_type_override = config.getString(config_prefix + ".handler.content_type");
|
||||
|
||||
std::shared_ptr<HandlingRuleHTTPHandlerFactory<PredefinedQueryHandler>> factory;
|
||||
|
||||
if (configuration.has(config_prefix + ".url"))
|
||||
if (config.has(config_prefix + ".url"))
|
||||
{
|
||||
auto url_expression = configuration.getString(config_prefix + ".url");
|
||||
auto url_expression = config.getString(config_prefix + ".url");
|
||||
|
||||
if (startsWith(url_expression, "regex:"))
|
||||
url_expression = url_expression.substr(6);
|
||||
@ -1247,7 +1249,7 @@ HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server, co
|
||||
std::move(regex),
|
||||
std::move(headers_name_with_regex),
|
||||
std::move(content_type_override));
|
||||
factory->addFiltersFromConfig(configuration, config_prefix);
|
||||
factory->addFiltersFromConfig(config, config_prefix);
|
||||
return factory;
|
||||
}
|
||||
}
|
||||
@ -1259,7 +1261,7 @@ HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server, co
|
||||
CompiledRegexPtr{},
|
||||
std::move(headers_name_with_regex),
|
||||
std::move(content_type_override));
|
||||
factory->addFiltersFromConfig(configuration, config_prefix);
|
||||
factory->addFiltersFromConfig(config, config_prefix);
|
||||
|
||||
return factory;
|
||||
}
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <Access/Credentials.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <Poco/Util/LayeredConfiguration.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
#include "HTTPHandler.h"
|
||||
#include "NotFoundHandler.h"
|
||||
@ -27,7 +27,11 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
static void addCommonDefaultHandlersFactory(HTTPRequestHandlerFactoryMain & factory, IServer & server);
|
||||
static void addDefaultHandlersFactory(HTTPRequestHandlerFactoryMain & factory, IServer & server, AsynchronousMetrics & async_metrics);
|
||||
static void addDefaultHandlersFactory(
|
||||
HTTPRequestHandlerFactoryMain & factory,
|
||||
IServer & server,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
AsynchronousMetrics & async_metrics);
|
||||
|
||||
HTTPRequestHandlerFactoryMain::HTTPRequestHandlerFactoryMain(const std::string & name_)
|
||||
: log(&Poco::Logger::get(name_)), name(name_)
|
||||
@ -59,37 +63,41 @@ std::unique_ptr<HTTPRequestHandler> HTTPRequestHandlerFactoryMain::createRequest
|
||||
}
|
||||
|
||||
static inline auto createHandlersFactoryFromConfig(
|
||||
IServer & server, const std::string & name, const String & prefix, AsynchronousMetrics & async_metrics)
|
||||
IServer & server,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & name,
|
||||
const String & prefix,
|
||||
AsynchronousMetrics & async_metrics)
|
||||
{
|
||||
auto main_handler_factory = std::make_shared<HTTPRequestHandlerFactoryMain>(name);
|
||||
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
server.config().keys(prefix, keys);
|
||||
config.keys(prefix, keys);
|
||||
|
||||
for (const auto & key : keys)
|
||||
{
|
||||
if (key == "defaults")
|
||||
{
|
||||
addDefaultHandlersFactory(*main_handler_factory, server, async_metrics);
|
||||
addDefaultHandlersFactory(*main_handler_factory, server, config, async_metrics);
|
||||
}
|
||||
else if (startsWith(key, "rule"))
|
||||
{
|
||||
const auto & handler_type = server.config().getString(prefix + "." + key + ".handler.type", "");
|
||||
const auto & handler_type = config.getString(prefix + "." + key + ".handler.type", "");
|
||||
|
||||
if (handler_type.empty())
|
||||
throw Exception("Handler type in config is not specified here: " + prefix + "." + key + ".handler.type",
|
||||
ErrorCodes::INVALID_CONFIG_PARAMETER);
|
||||
|
||||
if (handler_type == "static")
|
||||
main_handler_factory->addHandler(createStaticHandlerFactory(server, prefix + "." + key));
|
||||
main_handler_factory->addHandler(createStaticHandlerFactory(server, config, prefix + "." + key));
|
||||
else if (handler_type == "dynamic_query_handler")
|
||||
main_handler_factory->addHandler(createDynamicHandlerFactory(server, prefix + "." + key));
|
||||
main_handler_factory->addHandler(createDynamicHandlerFactory(server, config, prefix + "." + key));
|
||||
else if (handler_type == "predefined_query_handler")
|
||||
main_handler_factory->addHandler(createPredefinedHandlerFactory(server, prefix + "." + key));
|
||||
main_handler_factory->addHandler(createPredefinedHandlerFactory(server, config, prefix + "." + key));
|
||||
else if (handler_type == "prometheus")
|
||||
main_handler_factory->addHandler(createPrometheusHandlerFactory(server, async_metrics, prefix + "." + key));
|
||||
main_handler_factory->addHandler(createPrometheusHandlerFactory(server, config, async_metrics, prefix + "." + key));
|
||||
else if (handler_type == "replicas_status")
|
||||
main_handler_factory->addHandler(createReplicasStatusHandlerFactory(server, prefix + "." + key));
|
||||
main_handler_factory->addHandler(createReplicasStatusHandlerFactory(server, config, prefix + "." + key));
|
||||
else
|
||||
throw Exception("Unknown handler type '" + handler_type + "' in config here: " + prefix + "." + key + ".handler.type",
|
||||
ErrorCodes::INVALID_CONFIG_PARAMETER);
|
||||
@ -103,16 +111,16 @@ static inline auto createHandlersFactoryFromConfig(
|
||||
}
|
||||
|
||||
static inline HTTPRequestHandlerFactoryPtr
|
||||
createHTTPHandlerFactory(IServer & server, const std::string & name, AsynchronousMetrics & async_metrics)
|
||||
createHTTPHandlerFactory(IServer & server, const Poco::Util::AbstractConfiguration & config, const std::string & name, AsynchronousMetrics & async_metrics)
|
||||
{
|
||||
if (server.config().has("http_handlers"))
|
||||
if (config.has("http_handlers"))
|
||||
{
|
||||
return createHandlersFactoryFromConfig(server, name, "http_handlers", async_metrics);
|
||||
return createHandlersFactoryFromConfig(server, config, name, "http_handlers", async_metrics);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto factory = std::make_shared<HTTPRequestHandlerFactoryMain>(name);
|
||||
addDefaultHandlersFactory(*factory, server, async_metrics);
|
||||
addDefaultHandlersFactory(*factory, server, config, async_metrics);
|
||||
return factory;
|
||||
}
|
||||
}
|
||||
@ -129,18 +137,18 @@ static inline HTTPRequestHandlerFactoryPtr createInterserverHTTPHandlerFactory(I
|
||||
return factory;
|
||||
}
|
||||
|
||||
HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server, AsynchronousMetrics & async_metrics, const std::string & name)
|
||||
HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server, const Poco::Util::AbstractConfiguration & config, AsynchronousMetrics & async_metrics, const std::string & name)
|
||||
{
|
||||
if (name == "HTTPHandler-factory" || name == "HTTPSHandler-factory")
|
||||
return createHTTPHandlerFactory(server, name, async_metrics);
|
||||
return createHTTPHandlerFactory(server, config, name, async_metrics);
|
||||
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(server.config(), "prometheus", async_metrics));
|
||||
handler->attachStrictPath(server.config().getString("prometheus.endpoint", "/metrics"));
|
||||
server, PrometheusMetricsWriter(config, "prometheus", async_metrics));
|
||||
handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics"));
|
||||
handler->allowGetAndHeadRequest();
|
||||
factory->addHandler(handler);
|
||||
return factory;
|
||||
@ -185,7 +193,11 @@ void addCommonDefaultHandlersFactory(HTTPRequestHandlerFactoryMain & factory, IS
|
||||
factory.addHandler(js_handler);
|
||||
}
|
||||
|
||||
void addDefaultHandlersFactory(HTTPRequestHandlerFactoryMain & factory, IServer & server, AsynchronousMetrics & async_metrics)
|
||||
void addDefaultHandlersFactory(
|
||||
HTTPRequestHandlerFactoryMain & factory,
|
||||
IServer & server,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
AsynchronousMetrics & async_metrics)
|
||||
{
|
||||
addCommonDefaultHandlersFactory(factory, server);
|
||||
|
||||
@ -195,11 +207,11 @@ void addDefaultHandlersFactory(HTTPRequestHandlerFactoryMain & factory, IServer
|
||||
|
||||
/// We check that prometheus handler will be served on current (default) port.
|
||||
/// Otherwise it will be created separately, see createHandlerFactory(...).
|
||||
if (server.config().has("prometheus") && server.config().getInt("prometheus.port", 0) == 0)
|
||||
if (config.has("prometheus") && config.getInt("prometheus.port", 0) == 0)
|
||||
{
|
||||
auto prometheus_handler = std::make_shared<HandlingRuleHTTPHandlerFactory<PrometheusRequestHandler>>(
|
||||
server, PrometheusMetricsWriter(server.config(), "prometheus", async_metrics));
|
||||
prometheus_handler->attachStrictPath(server.config().getString("prometheus.endpoint", "/metrics"));
|
||||
server, PrometheusMetricsWriter(config, "prometheus", async_metrics));
|
||||
prometheus_handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics"));
|
||||
prometheus_handler->allowGetAndHeadRequest();
|
||||
factory.addHandler(prometheus_handler);
|
||||
}
|
||||
|
@ -7,7 +7,7 @@
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
#include <Poco/Util/LayeredConfiguration.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -63,7 +63,7 @@ public:
|
||||
};
|
||||
}
|
||||
|
||||
void addFiltersFromConfig(Poco::Util::AbstractConfiguration & config, const std::string & prefix)
|
||||
void addFiltersFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & prefix)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys filters_type;
|
||||
config.keys(prefix, filters_type);
|
||||
@ -126,16 +126,34 @@ private:
|
||||
std::function<std::unique_ptr<HTTPRequestHandler> ()> creator;
|
||||
};
|
||||
|
||||
HTTPRequestHandlerFactoryPtr createStaticHandlerFactory(IServer & server, const std::string & config_prefix);
|
||||
HTTPRequestHandlerFactoryPtr createStaticHandlerFactory(IServer & server,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix);
|
||||
|
||||
HTTPRequestHandlerFactoryPtr createDynamicHandlerFactory(IServer & server, const std::string & config_prefix);
|
||||
HTTPRequestHandlerFactoryPtr createDynamicHandlerFactory(IServer & server,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix);
|
||||
|
||||
HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server, const std::string & config_prefix);
|
||||
HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix);
|
||||
|
||||
HTTPRequestHandlerFactoryPtr createReplicasStatusHandlerFactory(IServer & server, const std::string & config_prefix);
|
||||
HTTPRequestHandlerFactoryPtr createReplicasStatusHandlerFactory(IServer & server,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix);
|
||||
|
||||
HTTPRequestHandlerFactoryPtr
|
||||
createPrometheusHandlerFactory(IServer & server, AsynchronousMetrics & async_metrics, const std::string & config_prefix);
|
||||
createPrometheusHandlerFactory(IServer & server,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
AsynchronousMetrics & async_metrics,
|
||||
const std::string & config_prefix);
|
||||
|
||||
/// @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)
|
||||
HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
AsynchronousMetrics & async_metrics,
|
||||
const std::string & name);
|
||||
|
||||
HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server, AsynchronousMetrics & async_metrics, const std::string & name);
|
||||
}
|
||||
|
@ -39,7 +39,7 @@ static inline bool checkExpression(std::string_view match_str, const std::pair<S
|
||||
return match_str == expression.first;
|
||||
}
|
||||
|
||||
static inline auto methodsFilter(Poco::Util::AbstractConfiguration & config, const std::string & config_path) /// NOLINT
|
||||
static inline auto methodsFilter(const Poco::Util::AbstractConfiguration & config, const std::string & config_path) /// NOLINT
|
||||
{
|
||||
std::vector<String> methods;
|
||||
Poco::StringTokenizer tokenizer(config.getString(config_path), ",");
|
||||
@ -63,7 +63,7 @@ static inline auto getExpression(const std::string & expression)
|
||||
return std::make_pair(expression, compiled_regex);
|
||||
}
|
||||
|
||||
static inline auto urlFilter(Poco::Util::AbstractConfiguration & config, const std::string & config_path) /// NOLINT
|
||||
static inline auto urlFilter(const Poco::Util::AbstractConfiguration & config, const std::string & config_path) /// NOLINT
|
||||
{
|
||||
return [expression = getExpression(config.getString(config_path))](const HTTPServerRequest & request)
|
||||
{
|
||||
@ -74,7 +74,7 @@ static inline auto urlFilter(Poco::Util::AbstractConfiguration & config, const s
|
||||
};
|
||||
}
|
||||
|
||||
static inline auto headersFilter(Poco::Util::AbstractConfiguration & config, const std::string & prefix) /// NOLINT
|
||||
static inline auto headersFilter(const Poco::Util::AbstractConfiguration & config, const std::string & prefix) /// NOLINT
|
||||
{
|
||||
std::unordered_map<String, std::pair<String, CompiledRegexPtr>> headers_expression;
|
||||
Poco::Util::AbstractConfiguration::Keys headers_name;
|
||||
|
@ -42,11 +42,14 @@ void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPSe
|
||||
}
|
||||
|
||||
HTTPRequestHandlerFactoryPtr
|
||||
createPrometheusHandlerFactory(IServer & server, AsynchronousMetrics & async_metrics, const std::string & config_prefix)
|
||||
createPrometheusHandlerFactory(IServer & server,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
AsynchronousMetrics & async_metrics,
|
||||
const std::string & config_prefix)
|
||||
{
|
||||
auto factory = std::make_shared<HandlingRuleHTTPHandlerFactory<PrometheusRequestHandler>>(
|
||||
server, PrometheusMetricsWriter(server.config(), config_prefix + ".handler", async_metrics));
|
||||
factory->addFiltersFromConfig(server.config(), config_prefix);
|
||||
server, PrometheusMetricsWriter(config, config_prefix + ".handler", async_metrics));
|
||||
factory->addFiltersFromConfig(config, config_prefix);
|
||||
return factory;
|
||||
}
|
||||
|
||||
|
@ -108,10 +108,12 @@ void ReplicasStatusHandler::handleRequest(HTTPServerRequest & request, HTTPServe
|
||||
}
|
||||
}
|
||||
|
||||
HTTPRequestHandlerFactoryPtr createReplicasStatusHandlerFactory(IServer & server, const std::string & config_prefix)
|
||||
HTTPRequestHandlerFactoryPtr createReplicasStatusHandlerFactory(IServer & server,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix)
|
||||
{
|
||||
auto factory = std::make_shared<HandlingRuleHTTPHandlerFactory<ReplicasStatusHandler>>(server);
|
||||
factory->addFiltersFromConfig(server.config(), config_prefix);
|
||||
factory->addFiltersFromConfig(config, config_prefix);
|
||||
return factory;
|
||||
}
|
||||
|
||||
|
@ -169,15 +169,17 @@ StaticRequestHandler::StaticRequestHandler(IServer & server_, const String & exp
|
||||
{
|
||||
}
|
||||
|
||||
HTTPRequestHandlerFactoryPtr createStaticHandlerFactory(IServer & server, const std::string & config_prefix)
|
||||
HTTPRequestHandlerFactoryPtr createStaticHandlerFactory(IServer & server,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix)
|
||||
{
|
||||
int status = server.config().getInt(config_prefix + ".handler.status", 200);
|
||||
std::string response_content = server.config().getRawString(config_prefix + ".handler.response_content", "Ok.\n");
|
||||
std::string response_content_type = server.config().getString(config_prefix + ".handler.content_type", "text/plain; charset=UTF-8");
|
||||
int status = config.getInt(config_prefix + ".handler.status", 200);
|
||||
std::string response_content = config.getRawString(config_prefix + ".handler.response_content", "Ok.\n");
|
||||
std::string response_content_type = config.getString(config_prefix + ".handler.content_type", "text/plain; charset=UTF-8");
|
||||
auto factory = std::make_shared<HandlingRuleHTTPHandlerFactory<StaticRequestHandler>>(
|
||||
server, std::move(response_content), std::move(status), std::move(response_content_type));
|
||||
|
||||
factory->addFiltersFromConfig(server.config(), config_prefix);
|
||||
factory->addFiltersFromConfig(config, config_prefix);
|
||||
|
||||
return factory;
|
||||
}
|
||||
|
@ -5,12 +5,29 @@
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
/**
|
||||
* When ClickHouse has frozen data on remote storage it required 'smart' data removing during UNFREEZE.
|
||||
* For remote storage actually frozen not remote data but local metadata with referrers on remote data.
|
||||
* So remote data can be referred from working and frozen data sets (or two frozen) at same time.
|
||||
* In this case during UNFREEZE ClickHouse should remove only local metadata and keep remote data.
|
||||
* But when data was already removed from working data set ClickHouse should remove remote data too.
|
||||
* To detect is current data used or not in some other place ClickHouse uses
|
||||
* - ref_count from metadata to check if data used in some other metadata on the same replica;
|
||||
* - Keeper record to check if data used on other replica.
|
||||
* StorageReplicatedMergeTree::removeSharedDetachedPart makes required checks, so here this method
|
||||
* called for each frozen part.
|
||||
*/
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int SUPPORT_IS_DISABLED;
|
||||
}
|
||||
|
||||
void FreezeMetaData::fill(const StorageReplicatedMergeTree & storage)
|
||||
{
|
||||
is_replicated = storage.supportsReplication();
|
||||
is_remote = storage.isRemote();
|
||||
replica_name = storage.getReplicaName();
|
||||
zookeeper_name = storage.getZooKeeperName();
|
||||
table_shared_id = storage.getTableSharedID();
|
||||
@ -26,11 +43,17 @@ void FreezeMetaData::save(DiskPtr data_disk, const String & path) const
|
||||
|
||||
writeIntText(version, buffer);
|
||||
buffer.write("\n", 1);
|
||||
writeBoolText(is_replicated, buffer);
|
||||
buffer.write("\n", 1);
|
||||
writeBoolText(is_remote, buffer);
|
||||
buffer.write("\n", 1);
|
||||
writeString(replica_name, buffer);
|
||||
if (version == 1)
|
||||
{
|
||||
/// is_replicated and is_remote are not used
|
||||
bool is_replicated = true;
|
||||
writeBoolText(is_replicated, buffer);
|
||||
buffer.write("\n", 1);
|
||||
bool is_remote = true;
|
||||
writeBoolText(is_remote, buffer);
|
||||
buffer.write("\n", 1);
|
||||
}
|
||||
writeString(escapeForFileName(replica_name), buffer);
|
||||
buffer.write("\n", 1);
|
||||
writeString(zookeeper_name, buffer);
|
||||
buffer.write("\n", 1);
|
||||
@ -51,17 +74,25 @@ bool FreezeMetaData::load(DiskPtr data_disk, const String & path)
|
||||
auto metadata_str = metadata_storage->readFileToString(file_path);
|
||||
ReadBufferFromString buffer(metadata_str);
|
||||
readIntText(version, buffer);
|
||||
if (version != 1)
|
||||
if (version < 1 || version > 2)
|
||||
{
|
||||
LOG_ERROR(&Poco::Logger::get("FreezeMetaData"), "Unknown freezed metadata version: {}", version);
|
||||
LOG_ERROR(&Poco::Logger::get("FreezeMetaData"), "Unknown frozen metadata version: {}", version);
|
||||
return false;
|
||||
}
|
||||
DB::assertChar('\n', buffer);
|
||||
readBoolText(is_replicated, buffer);
|
||||
DB::assertChar('\n', buffer);
|
||||
readBoolText(is_remote, buffer);
|
||||
DB::assertChar('\n', buffer);
|
||||
readString(replica_name, buffer);
|
||||
if (version == 1)
|
||||
{
|
||||
/// is_replicated and is_remote are not used
|
||||
bool is_replicated;
|
||||
readBoolText(is_replicated, buffer);
|
||||
DB::assertChar('\n', buffer);
|
||||
bool is_remote;
|
||||
readBoolText(is_remote, buffer);
|
||||
DB::assertChar('\n', buffer);
|
||||
}
|
||||
std::string unescaped_replica_name;
|
||||
readString(unescaped_replica_name, buffer);
|
||||
replica_name = unescapeForFileName(unescaped_replica_name);
|
||||
DB::assertChar('\n', buffer);
|
||||
readString(zookeeper_name, buffer);
|
||||
DB::assertChar('\n', buffer);
|
||||
@ -87,9 +118,23 @@ String FreezeMetaData::getFileName(const String & path)
|
||||
return fs::path(path) / "frozen_metadata.txt";
|
||||
}
|
||||
|
||||
BlockIO Unfreezer::unfreeze(const String & backup_name, ContextPtr local_context)
|
||||
Unfreezer::Unfreezer(ContextPtr context) : local_context(context)
|
||||
{
|
||||
LOG_DEBUG(log, "Unfreezing backup {}", backup_name);
|
||||
if (local_context->hasZooKeeper())
|
||||
zookeeper = local_context->getZooKeeper();
|
||||
}
|
||||
|
||||
BlockIO Unfreezer::systemUnfreeze(const String & backup_name)
|
||||
{
|
||||
LOG_DEBUG(log, "Unfreezing backup {}", escapeForFileName(backup_name));
|
||||
|
||||
const auto & config = local_context->getConfigRef();
|
||||
static constexpr auto config_key = "enable_system_unfreeze";
|
||||
if (!config.getBool(config_key, false))
|
||||
{
|
||||
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Support for SYSTEM UNFREEZE query is disabled. You can enable it via '{}' server setting", config_key);
|
||||
}
|
||||
|
||||
auto disks_map = local_context->getDisksMap();
|
||||
Disks disks;
|
||||
for (auto & [name, disk]: disks_map)
|
||||
@ -97,33 +142,38 @@ BlockIO Unfreezer::unfreeze(const String & backup_name, ContextPtr local_context
|
||||
disks.push_back(disk);
|
||||
}
|
||||
auto backup_path = fs::path(backup_directory_prefix) / escapeForFileName(backup_name);
|
||||
auto store_path = backup_path / "store";
|
||||
auto store_paths = {backup_path / "store", backup_path / "data"};
|
||||
|
||||
PartitionCommandsResultInfo result_info;
|
||||
|
||||
for (const auto & disk: disks)
|
||||
{
|
||||
if (!disk->exists(store_path))
|
||||
continue;
|
||||
for (auto prefix_it = disk->iterateDirectory(store_path); prefix_it->isValid(); prefix_it->next())
|
||||
for (const auto& store_path: store_paths)
|
||||
{
|
||||
auto prefix_directory = store_path / prefix_it->name();
|
||||
for (auto table_it = disk->iterateDirectory(prefix_directory); table_it->isValid(); table_it->next())
|
||||
if (!disk->exists(store_path))
|
||||
continue;
|
||||
for (auto prefix_it = disk->iterateDirectory(store_path); prefix_it->isValid(); prefix_it->next())
|
||||
{
|
||||
auto table_directory = prefix_directory / table_it->name();
|
||||
auto current_result_info = unfreezePartitionsFromTableDirectory([] (const String &) { return true; }, backup_name, {disk}, table_directory, local_context);
|
||||
for (auto & command_result : current_result_info)
|
||||
auto prefix_directory = store_path / prefix_it->name();
|
||||
for (auto table_it = disk->iterateDirectory(prefix_directory); table_it->isValid(); table_it->next())
|
||||
{
|
||||
command_result.command_type = "SYSTEM UNFREEZE";
|
||||
auto table_directory = prefix_directory / table_it->name();
|
||||
auto current_result_info = unfreezePartitionsFromTableDirectory(
|
||||
[](const String &) { return true; }, backup_name, {disk}, table_directory);
|
||||
for (auto & command_result : current_result_info)
|
||||
{
|
||||
command_result.command_type = "SYSTEM UNFREEZE";
|
||||
}
|
||||
result_info.insert(
|
||||
result_info.end(),
|
||||
std::make_move_iterator(current_result_info.begin()),
|
||||
std::make_move_iterator(current_result_info.end()));
|
||||
}
|
||||
result_info.insert(
|
||||
result_info.end(),
|
||||
std::make_move_iterator(current_result_info.begin()),
|
||||
std::make_move_iterator(current_result_info.end()));
|
||||
}
|
||||
}
|
||||
if (disk->exists(backup_path))
|
||||
{
|
||||
/// After unfreezing we need to clear revision.txt file and empty directories
|
||||
disk->removeRecursive(backup_path);
|
||||
}
|
||||
}
|
||||
@ -136,18 +186,15 @@ BlockIO Unfreezer::unfreeze(const String & backup_name, ContextPtr local_context
|
||||
return result;
|
||||
}
|
||||
|
||||
bool Unfreezer::removeFreezedPart(DiskPtr disk, const String & path, const String & part_name, ContextPtr local_context)
|
||||
bool Unfreezer::removeFreezedPart(DiskPtr disk, const String & path, const String & part_name, ContextPtr local_context, zkutil::ZooKeeperPtr zookeeper)
|
||||
{
|
||||
if (disk->supportZeroCopyReplication())
|
||||
{
|
||||
FreezeMetaData meta;
|
||||
if (meta.load(disk, path))
|
||||
{
|
||||
if (meta.is_replicated)
|
||||
{
|
||||
FreezeMetaData::clean(disk, path);
|
||||
return StorageReplicatedMergeTree::removeSharedDetachedPart(disk, path, part_name, meta.table_shared_id, meta.zookeeper_name, meta.replica_name, "", local_context);
|
||||
}
|
||||
FreezeMetaData::clean(disk, path);
|
||||
return StorageReplicatedMergeTree::removeSharedDetachedPart(disk, path, part_name, meta.table_shared_id, meta.zookeeper_name, meta.replica_name, "", local_context, zookeeper);
|
||||
}
|
||||
}
|
||||
|
||||
@ -156,7 +203,7 @@ bool Unfreezer::removeFreezedPart(DiskPtr disk, const String & path, const Strin
|
||||
return false;
|
||||
}
|
||||
|
||||
PartitionCommandsResultInfo Unfreezer::unfreezePartitionsFromTableDirectory(MergeTreeData::MatcherFn matcher, const String & backup_name, const Disks & disks, const fs::path & table_directory, ContextPtr local_context)
|
||||
PartitionCommandsResultInfo Unfreezer::unfreezePartitionsFromTableDirectory(MergeTreeData::MatcherFn matcher, const String & backup_name, const Disks & disks, const fs::path & table_directory)
|
||||
{
|
||||
PartitionCommandsResultInfo result;
|
||||
|
||||
@ -180,7 +227,7 @@ PartitionCommandsResultInfo Unfreezer::unfreezePartitionsFromTableDirectory(Merg
|
||||
|
||||
const auto & path = it->path();
|
||||
|
||||
bool keep_shared = removeFreezedPart(disk, path, partition_directory, local_context);
|
||||
bool keep_shared = removeFreezedPart(disk, path, partition_directory, local_context, zookeeper);
|
||||
|
||||
result.push_back(PartitionCommandResultInfo{
|
||||
.partition_id = partition_id,
|
||||
|
@ -23,9 +23,7 @@ private:
|
||||
static String getFileName(const String & path);
|
||||
|
||||
public:
|
||||
int version = 1;
|
||||
bool is_replicated{false};
|
||||
bool is_remote{false};
|
||||
int version = 2;
|
||||
String replica_name;
|
||||
String zookeeper_name;
|
||||
String table_shared_id;
|
||||
@ -34,12 +32,15 @@ public:
|
||||
class Unfreezer
|
||||
{
|
||||
public:
|
||||
PartitionCommandsResultInfo unfreezePartitionsFromTableDirectory(MergeTreeData::MatcherFn matcher, const String & backup_name, const Disks & disks, const fs::path & table_directory, ContextPtr local_context);
|
||||
BlockIO unfreeze(const String & backup_name, ContextPtr local_context);
|
||||
Unfreezer(ContextPtr context);
|
||||
PartitionCommandsResultInfo unfreezePartitionsFromTableDirectory(MergeTreeData::MatcherFn matcher, const String & backup_name, const Disks & disks, const fs::path & table_directory);
|
||||
BlockIO systemUnfreeze(const String & backup_name);
|
||||
private:
|
||||
ContextPtr local_context;
|
||||
zkutil::ZooKeeperPtr zookeeper;
|
||||
Poco::Logger * log = &Poco::Logger::get("Unfreezer");
|
||||
static constexpr std::string_view backup_directory_prefix = "shadow";
|
||||
static bool removeFreezedPart(DiskPtr disk, const String & path, const String & part_name, ContextPtr local_context);
|
||||
static bool removeFreezedPart(DiskPtr disk, const String & path, const String & part_name, ContextPtr local_context, zkutil::ZooKeeperPtr zookeeper);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1992,7 +1992,7 @@ size_t MergeTreeData::clearOldBrokenPartsFromDetachedDirectory()
|
||||
|
||||
for (auto & [old_name, new_name, disk] : renamed_parts.old_and_new_names)
|
||||
{
|
||||
removeDetachedPart(disk, fs::path(relative_data_path) / "detached" / new_name / "", old_name, false);
|
||||
removeDetachedPart(disk, fs::path(relative_data_path) / "detached" / new_name / "", old_name);
|
||||
LOG_DEBUG(log, "Removed broken detached part {} due to a timeout for broken detached parts", old_name);
|
||||
old_name.clear();
|
||||
}
|
||||
@ -4744,7 +4744,7 @@ void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, ContextPtr
|
||||
|
||||
for (auto & [old_name, new_name, disk] : renamed_parts.old_and_new_names)
|
||||
{
|
||||
bool keep_shared = removeDetachedPart(disk, fs::path(relative_data_path) / "detached" / new_name / "", old_name, false);
|
||||
bool keep_shared = removeDetachedPart(disk, fs::path(relative_data_path) / "detached" / new_name / "", old_name);
|
||||
LOG_DEBUG(log, "Dropped detached part {}, keep shared data: {}", old_name, keep_shared);
|
||||
old_name.clear();
|
||||
}
|
||||
@ -6411,7 +6411,7 @@ PartitionCommandsResultInfo MergeTreeData::unfreezeAll(
|
||||
return unfreezePartitionsByMatcher([] (const String &) { return true; }, backup_name, local_context);
|
||||
}
|
||||
|
||||
bool MergeTreeData::removeDetachedPart(DiskPtr disk, const String & path, const String &, bool)
|
||||
bool MergeTreeData::removeDetachedPart(DiskPtr disk, const String & path, const String &)
|
||||
{
|
||||
disk->removeRecursive(path);
|
||||
|
||||
@ -6426,7 +6426,7 @@ PartitionCommandsResultInfo MergeTreeData::unfreezePartitionsByMatcher(MatcherFn
|
||||
|
||||
auto disks = getStoragePolicy()->getDisks();
|
||||
|
||||
return Unfreezer().unfreezePartitionsFromTableDirectory(matcher, backup_name, disks, backup_path, local_context);
|
||||
return Unfreezer(local_context).unfreezePartitionsFromTableDirectory(matcher, backup_name, disks, backup_path);
|
||||
}
|
||||
|
||||
bool MergeTreeData::canReplacePartition(const DataPartPtr & src_part) const
|
||||
|
@ -970,7 +970,7 @@ public:
|
||||
|
||||
/// Check shared data usage on other replicas for detached/freezed part
|
||||
/// Remove local files and remote files if needed
|
||||
virtual bool removeDetachedPart(DiskPtr disk, const String & path, const String & part_name, bool is_freezed);
|
||||
virtual bool removeDetachedPart(DiskPtr disk, const String & path, const String & part_name);
|
||||
|
||||
virtual String getTableSharedID() const { return ""; }
|
||||
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <Storages/MergeTree/MergeTreeReaderCompact.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartWriterCompact.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h>
|
||||
|
||||
|
||||
@ -47,9 +48,11 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader(
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback) const
|
||||
{
|
||||
auto read_info = std::make_shared<LoadedMergeTreeDataPartInfoForReader>(shared_from_this());
|
||||
auto * load_marks_threadpool = reader_settings.read_settings.load_marks_asynchronously ? &read_info->getContext()->getLoadMarksThreadpool() : nullptr;
|
||||
|
||||
return std::make_unique<MergeTreeReaderCompact>(
|
||||
read_info, columns_to_read, metadata_snapshot, uncompressed_cache,
|
||||
mark_cache, mark_ranges, reader_settings,
|
||||
mark_cache, mark_ranges, reader_settings, load_marks_threadpool,
|
||||
avg_value_size_hints, profile_callback);
|
||||
}
|
||||
|
||||
@ -93,7 +96,7 @@ void MergeTreeDataPartCompact::calculateEachColumnSizes(ColumnSizeByName & /*eac
|
||||
|
||||
void MergeTreeDataPartCompact::loadIndexGranularityImpl(
|
||||
MergeTreeIndexGranularity & index_granularity_, const MergeTreeIndexGranularityInfo & index_granularity_info_,
|
||||
const NamesAndTypesList & columns_, const DataPartStoragePtr & data_part_storage_)
|
||||
size_t columns_count, const DataPartStoragePtr & data_part_storage_)
|
||||
{
|
||||
if (!index_granularity_info_.is_adaptive)
|
||||
throw Exception("MergeTreeDataPartCompact cannot be created with non-adaptive granulary.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
@ -111,13 +114,13 @@ void MergeTreeDataPartCompact::loadIndexGranularityImpl(
|
||||
while (!buffer->eof())
|
||||
{
|
||||
/// Skip offsets for columns
|
||||
buffer->seek(columns_.size() * sizeof(MarkInCompressedFile), SEEK_CUR);
|
||||
buffer->seek(columns_count * sizeof(MarkInCompressedFile), SEEK_CUR);
|
||||
size_t granularity;
|
||||
readIntBinary(granularity, *buffer);
|
||||
index_granularity_.appendMark(granularity);
|
||||
}
|
||||
|
||||
if (index_granularity_.getMarksCount() * index_granularity_info_.getMarkSizeInBytes(columns_.size()) != marks_file_size)
|
||||
if (index_granularity_.getMarksCount() * index_granularity_info_.getMarkSizeInBytes(columns_count) != marks_file_size)
|
||||
throw Exception("Cannot read all marks from file " + marks_file_path, ErrorCodes::CANNOT_READ_ALL_DATA);
|
||||
|
||||
index_granularity_.setInitialized();
|
||||
@ -128,7 +131,7 @@ void MergeTreeDataPartCompact::loadIndexGranularity()
|
||||
if (columns.empty())
|
||||
throw Exception("No columns in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
|
||||
loadIndexGranularityImpl(index_granularity, index_granularity_info, columns, data_part_storage);
|
||||
loadIndexGranularityImpl(index_granularity, index_granularity_info, columns.size(), data_part_storage);
|
||||
}
|
||||
|
||||
bool MergeTreeDataPartCompact::hasColumnFiles(const NameAndTypePair & column) const
|
||||
|
@ -68,7 +68,7 @@ public:
|
||||
protected:
|
||||
static void loadIndexGranularityImpl(
|
||||
MergeTreeIndexGranularity & index_granularity_, const MergeTreeIndexGranularityInfo & index_granularity_info_,
|
||||
const NamesAndTypesList & columns_, const DataPartStoragePtr & data_part_storage_);
|
||||
size_t columns_count, const DataPartStoragePtr & data_part_storage_);
|
||||
|
||||
private:
|
||||
void checkConsistency(bool require_part_metadata) const override;
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Storages/MergeTree/MergeTreeIndexReader.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace
|
||||
{
|
||||
@ -15,6 +16,9 @@ std::unique_ptr<MergeTreeReaderStream> makeIndexReader(
|
||||
UncompressedCache * uncompressed_cache,
|
||||
MergeTreeReaderSettings settings)
|
||||
{
|
||||
auto context = part->storage.getContext();
|
||||
auto * load_marks_threadpool = settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr;
|
||||
|
||||
return std::make_unique<MergeTreeReaderStream>(
|
||||
part->data_part_storage,
|
||||
index->getFileName(), extension, marks_count,
|
||||
@ -22,7 +26,7 @@ std::unique_ptr<MergeTreeReaderStream> makeIndexReader(
|
||||
std::move(settings), mark_cache, uncompressed_cache,
|
||||
part->getFileSizeOrZero(index->getFileName() + extension),
|
||||
&part->index_granularity_info,
|
||||
ReadBufferFromFileBase::ProfileCallback{}, CLOCK_MONOTONIC_COARSE, false);
|
||||
ReadBufferFromFileBase::ProfileCallback{}, CLOCK_MONOTONIC_COARSE, false, load_marks_threadpool);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,9 +2,19 @@
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Common/MemoryTrackerBlockerInThread.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/scope_guard_safe.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
|
||||
#include <utility>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event WaitMarksLoadMicroseconds;
|
||||
extern const Event BackgroundLoadingMarksTasks;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -23,6 +33,7 @@ MergeTreeMarksLoader::MergeTreeMarksLoader(
|
||||
const MergeTreeIndexGranularityInfo & index_granularity_info_,
|
||||
bool save_marks_in_cache_,
|
||||
const ReadSettings & read_settings_,
|
||||
ThreadPool * load_marks_threadpool_,
|
||||
size_t columns_in_mark_)
|
||||
: data_part_storage(std::move(data_part_storage_))
|
||||
, mark_cache(mark_cache_)
|
||||
@ -32,13 +43,41 @@ MergeTreeMarksLoader::MergeTreeMarksLoader(
|
||||
, save_marks_in_cache(save_marks_in_cache_)
|
||||
, columns_in_mark(columns_in_mark_)
|
||||
, read_settings(read_settings_)
|
||||
, load_marks_threadpool(load_marks_threadpool_)
|
||||
{
|
||||
if (load_marks_threadpool)
|
||||
{
|
||||
future = loadMarksAsync();
|
||||
}
|
||||
}
|
||||
|
||||
MergeTreeMarksLoader::~MergeTreeMarksLoader()
|
||||
{
|
||||
if (future.valid())
|
||||
{
|
||||
future.wait();
|
||||
}
|
||||
}
|
||||
|
||||
const MarkInCompressedFile & MergeTreeMarksLoader::getMark(size_t row_index, size_t column_index)
|
||||
{
|
||||
if (!marks)
|
||||
loadMarks();
|
||||
{
|
||||
Stopwatch watch(CLOCK_MONOTONIC);
|
||||
|
||||
if (future.valid())
|
||||
{
|
||||
marks = future.get();
|
||||
future = {};
|
||||
}
|
||||
else
|
||||
{
|
||||
marks = loadMarks();
|
||||
}
|
||||
|
||||
watch.stop();
|
||||
ProfileEvents::increment(ProfileEvents::WaitMarksLoadMicroseconds, watch.elapsedMicroseconds());
|
||||
}
|
||||
|
||||
#ifndef NDEBUG
|
||||
if (column_index >= columns_in_mark)
|
||||
@ -95,28 +134,63 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl()
|
||||
return res;
|
||||
}
|
||||
|
||||
void MergeTreeMarksLoader::loadMarks()
|
||||
MarkCache::MappedPtr MergeTreeMarksLoader::loadMarks()
|
||||
{
|
||||
MarkCache::MappedPtr loaded_marks;
|
||||
|
||||
if (mark_cache)
|
||||
{
|
||||
auto key = mark_cache->hash(fs::path(data_part_storage->getFullPath()) / mrk_path);
|
||||
if (save_marks_in_cache)
|
||||
{
|
||||
auto callback = [this]{ return loadMarksImpl(); };
|
||||
marks = mark_cache->getOrSet(key, callback);
|
||||
loaded_marks = mark_cache->getOrSet(key, callback);
|
||||
}
|
||||
else
|
||||
{
|
||||
marks = mark_cache->get(key);
|
||||
if (!marks)
|
||||
marks = loadMarksImpl();
|
||||
loaded_marks = mark_cache->get(key);
|
||||
if (!loaded_marks)
|
||||
loaded_marks = loadMarksImpl();
|
||||
}
|
||||
}
|
||||
else
|
||||
marks = loadMarksImpl();
|
||||
loaded_marks = loadMarksImpl();
|
||||
|
||||
if (!marks)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to load marks: {}", String(fs::path(data_part_storage->getFullPath()) / mrk_path));
|
||||
if (!loaded_marks)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR, "Failed to load marks: {}",
|
||||
(fs::path(data_part_storage->getFullPath()) / mrk_path).string());
|
||||
}
|
||||
|
||||
return loaded_marks;
|
||||
}
|
||||
|
||||
std::future<MarkCache::MappedPtr> MergeTreeMarksLoader::loadMarksAsync()
|
||||
{
|
||||
ThreadGroupStatusPtr thread_group;
|
||||
if (CurrentThread::isInitialized() && CurrentThread::get().getThreadGroup())
|
||||
thread_group = CurrentThread::get().getThreadGroup();
|
||||
|
||||
auto task = std::make_shared<std::packaged_task<MarkCache::MappedPtr()>>([thread_group, this]
|
||||
{
|
||||
setThreadName("loadMarksThread");
|
||||
|
||||
if (thread_group)
|
||||
CurrentThread::attachTo(thread_group);
|
||||
|
||||
SCOPE_EXIT_SAFE({
|
||||
if (thread_group)
|
||||
CurrentThread::detachQuery();
|
||||
});
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::BackgroundLoadingMarksTasks);
|
||||
return loadMarks();
|
||||
});
|
||||
|
||||
auto task_future = task->get_future();
|
||||
load_marks_threadpool->scheduleOrThrow([task]{ (*task)(); });
|
||||
return task_future;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,11 +2,13 @@
|
||||
#include <Storages/MergeTree/IDataPartStorage.h>
|
||||
#include <Storages/MarkCache.h>
|
||||
#include <IO/ReadSettings.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct MergeTreeIndexGranularityInfo;
|
||||
class Threadpool;
|
||||
|
||||
class MergeTreeMarksLoader
|
||||
{
|
||||
@ -21,8 +23,11 @@ public:
|
||||
const MergeTreeIndexGranularityInfo & index_granularity_info_,
|
||||
bool save_marks_in_cache_,
|
||||
const ReadSettings & read_settings_,
|
||||
ThreadPool * load_marks_threadpool_,
|
||||
size_t columns_in_mark_ = 1);
|
||||
|
||||
~MergeTreeMarksLoader();
|
||||
|
||||
const MarkInCompressedFile & getMark(size_t row_index, size_t column_index = 0);
|
||||
|
||||
private:
|
||||
@ -36,8 +41,12 @@ private:
|
||||
MarkCache::MappedPtr marks;
|
||||
ReadSettings read_settings;
|
||||
|
||||
void loadMarks();
|
||||
MarkCache::MappedPtr loadMarks();
|
||||
std::future<MarkCache::MappedPtr> loadMarksAsync();
|
||||
MarkCache::MappedPtr loadMarksImpl();
|
||||
|
||||
std::future<MarkCache::MappedPtr> future;
|
||||
ThreadPool * load_marks_threadpool;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -22,6 +22,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact(
|
||||
MarkCache * mark_cache_,
|
||||
MarkRanges mark_ranges_,
|
||||
MergeTreeReaderSettings settings_,
|
||||
ThreadPool * load_marks_threadpool_,
|
||||
ValueSizeMap avg_value_size_hints_,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback_,
|
||||
clockid_t clock_type_)
|
||||
@ -42,6 +43,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact(
|
||||
data_part_info_for_read_->getIndexGranularityInfo(),
|
||||
settings.save_marks_in_cache,
|
||||
settings.read_settings,
|
||||
load_marks_threadpool_,
|
||||
data_part_info_for_read_->getColumns().size())
|
||||
{
|
||||
try
|
||||
|
@ -26,6 +26,7 @@ public:
|
||||
MarkCache * mark_cache_,
|
||||
MarkRanges mark_ranges_,
|
||||
MergeTreeReaderSettings settings_,
|
||||
ThreadPool * load_marks_threadpool_,
|
||||
ValueSizeMap avg_value_size_hints_ = {},
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback_ = {},
|
||||
clockid_t clock_type_ = CLOCK_MONOTONIC_COARSE);
|
||||
|
@ -16,14 +16,19 @@ namespace ErrorCodes
|
||||
|
||||
MergeTreeReaderStream::MergeTreeReaderStream(
|
||||
DataPartStoragePtr data_part_storage_,
|
||||
const String & path_prefix_, const String & data_file_extension_, size_t marks_count_,
|
||||
const String & path_prefix_,
|
||||
const String & data_file_extension_,
|
||||
size_t marks_count_,
|
||||
const MarkRanges & all_mark_ranges_,
|
||||
const MergeTreeReaderSettings & settings_,
|
||||
MarkCache * mark_cache_,
|
||||
UncompressedCache * uncompressed_cache_, size_t file_size_,
|
||||
UncompressedCache * uncompressed_cache_,
|
||||
size_t file_size_,
|
||||
const MergeTreeIndexGranularityInfo * index_granularity_info_,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback_, clockid_t clock_type_,
|
||||
bool is_low_cardinality_dictionary_)
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback_,
|
||||
clockid_t clock_type_,
|
||||
bool is_low_cardinality_dictionary_,
|
||||
ThreadPool * load_marks_cache_threadpool_)
|
||||
: settings(settings_)
|
||||
, profile_callback(profile_callback_)
|
||||
, clock_type(clock_type_)
|
||||
@ -45,7 +50,8 @@ MergeTreeReaderStream::MergeTreeReaderStream(
|
||||
marks_count,
|
||||
*index_granularity_info,
|
||||
save_marks_in_cache,
|
||||
settings.read_settings)
|
||||
settings.read_settings,
|
||||
load_marks_cache_threadpool_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -20,13 +20,19 @@ class MergeTreeReaderStream
|
||||
public:
|
||||
MergeTreeReaderStream(
|
||||
DataPartStoragePtr data_part_storage_,
|
||||
const String & path_prefix_, const String & data_file_extension_, size_t marks_count_,
|
||||
const String & path_prefix_,
|
||||
const String & data_file_extension_,
|
||||
size_t marks_count_,
|
||||
const MarkRanges & all_mark_ranges,
|
||||
const MergeTreeReaderSettings & settings_,
|
||||
MarkCache * mark_cache, UncompressedCache * uncompressed_cache,
|
||||
size_t file_size_, const MergeTreeIndexGranularityInfo * index_granularity_info_,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type,
|
||||
bool is_low_cardinality_dictionary_);
|
||||
MarkCache * mark_cache,
|
||||
UncompressedCache * uncompressed_cache,
|
||||
size_t file_size_,
|
||||
const MergeTreeIndexGranularityInfo * index_granularity_info_,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback,
|
||||
clockid_t clock_type,
|
||||
bool is_low_cardinality_dictionary_,
|
||||
ThreadPool * load_marks_cache_threadpool_);
|
||||
|
||||
void seekToMark(size_t index);
|
||||
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <DataTypes/DataTypeNested.h>
|
||||
#include <Interpreters/inplaceBlockConversions.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartWide.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
@ -186,12 +187,15 @@ void MergeTreeReaderWide::addStreams(
|
||||
has_any_stream = true;
|
||||
bool is_lc_dict = substream_path.size() > 1 && substream_path[substream_path.size() - 2].type == ISerialization::Substream::Type::DictionaryKeys;
|
||||
|
||||
auto context = data_part_info_for_read->getContext();
|
||||
auto * load_marks_threadpool = settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr;
|
||||
|
||||
streams.emplace(stream_name, std::make_unique<MergeTreeReaderStream>(
|
||||
data_part_info_for_read->getDataPartStorage(), stream_name, DATA_FILE_EXTENSION,
|
||||
data_part_info_for_read->getMarksCount(), all_mark_ranges, settings, mark_cache,
|
||||
uncompressed_cache, data_part_info_for_read->getFileSizeOrZero(stream_name + DATA_FILE_EXTENSION),
|
||||
&data_part_info_for_read->getIndexGranularityInfo(),
|
||||
profile_callback, clock_type, is_lc_dict));
|
||||
profile_callback, clock_type, is_lc_dict, load_marks_threadpool));
|
||||
};
|
||||
|
||||
serialization->enumerateStreams(callback);
|
||||
|
@ -523,8 +523,7 @@ Chunk StorageEmbeddedRocksDB::getByKeys(
|
||||
|
||||
Block StorageEmbeddedRocksDB::getSampleBlock(const Names &) const
|
||||
{
|
||||
auto metadata = getInMemoryMetadataPtr();
|
||||
return metadata ? metadata->getSampleBlock() : Block();
|
||||
return getInMemoryMetadataPtr()->getSampleBlock();
|
||||
}
|
||||
|
||||
Chunk StorageEmbeddedRocksDB::getBySerializedKeys(
|
||||
|
771
src/Storages/StorageKeeperMap.cpp
Normal file
771
src/Storages/StorageKeeperMap.cpp
Normal file
@ -0,0 +1,771 @@
|
||||
#include <Storages/StorageKeeperMap.h>
|
||||
|
||||
#include <Columns/ColumnString.h>
|
||||
|
||||
#include <Databases/DatabaseReplicated.h>
|
||||
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Core/UUID.h>
|
||||
#include <Core/ServerUUID.h>
|
||||
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
|
||||
#include <Processors/ISource.h>
|
||||
#include <Processors/Sinks/SinkToStorage.h>
|
||||
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Storages/KVStorageUtils.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
|
||||
#include <Common/Base64.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/ZooKeeper/IKeeper.h>
|
||||
#include <Common/ZooKeeper/KeeperException.h>
|
||||
#include <Common/ZooKeeper/Types.h>
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperConstants.h>
|
||||
|
||||
#include <base/types.h>
|
||||
|
||||
#include <boost/algorithm/string/classification.hpp>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int KEEPER_EXCEPTION;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int LIMIT_EXCEEDED;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
std::string formattedAST(const ASTPtr & ast)
|
||||
{
|
||||
if (!ast)
|
||||
return "";
|
||||
return serializeAST(*ast);
|
||||
}
|
||||
|
||||
void verifyTableId(const StorageID & table_id)
|
||||
{
|
||||
if (!table_id.hasUUID())
|
||||
{
|
||||
auto database = DatabaseCatalog::instance().getDatabase(table_id.database_name);
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"KeeperMap cannot be used with '{}' database because it uses {} engine. Please use Atomic or Replicated database",
|
||||
table_id.getDatabaseName(),
|
||||
database->getEngineName());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
class StorageKeeperMapSink : public SinkToStorage
|
||||
{
|
||||
StorageKeeperMap & storage;
|
||||
std::unordered_map<std::string, std::string> new_values;
|
||||
size_t primary_key_pos;
|
||||
|
||||
public:
|
||||
StorageKeeperMapSink(StorageKeeperMap & storage_, const StorageMetadataPtr & metadata_snapshot)
|
||||
: SinkToStorage(metadata_snapshot->getSampleBlock()), storage(storage_)
|
||||
{
|
||||
auto primary_key = storage.getPrimaryKey();
|
||||
assert(primary_key.size() == 1);
|
||||
primary_key_pos = getHeader().getPositionByName(primary_key[0]);
|
||||
}
|
||||
|
||||
std::string getName() const override { return "StorageKeeperMapSink"; }
|
||||
|
||||
void consume(Chunk chunk) override
|
||||
{
|
||||
auto rows = chunk.getNumRows();
|
||||
auto block = getHeader().cloneWithColumns(chunk.detachColumns());
|
||||
|
||||
WriteBufferFromOwnString wb_key;
|
||||
WriteBufferFromOwnString wb_value;
|
||||
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
{
|
||||
wb_key.restart();
|
||||
wb_value.restart();
|
||||
|
||||
size_t idx = 0;
|
||||
for (const auto & elem : block)
|
||||
{
|
||||
elem.type->getDefaultSerialization()->serializeBinary(*elem.column, i, idx == primary_key_pos ? wb_key : wb_value);
|
||||
++idx;
|
||||
}
|
||||
|
||||
auto key = base64Encode(wb_key.str(), /* url_encoding */ true);
|
||||
new_values[std::move(key)] = std::move(wb_value.str());
|
||||
}
|
||||
}
|
||||
|
||||
void onFinish() override
|
||||
{
|
||||
auto zookeeper = storage.getClient();
|
||||
|
||||
Coordination::Requests requests;
|
||||
|
||||
auto keys_limit = storage.keysLimit();
|
||||
|
||||
size_t current_keys_num = 0;
|
||||
size_t new_keys_num = 0;
|
||||
|
||||
// We use keys limit as a soft limit so we ignore some cases when it can be still exceeded
|
||||
// (e.g if parallel insert queries are being run)
|
||||
if (keys_limit != 0)
|
||||
{
|
||||
Coordination::Stat data_stat;
|
||||
zookeeper->get(storage.dataPath(), &data_stat);
|
||||
current_keys_num = data_stat.numChildren;
|
||||
}
|
||||
|
||||
std::vector<std::pair<const std::string *, std::future<Coordination::ExistsResponse>>> exist_responses;
|
||||
for (const auto & [key, value] : new_values)
|
||||
{
|
||||
auto path = storage.fullPathForKey(key);
|
||||
|
||||
exist_responses.push_back({&key, zookeeper->asyncExists(path)});
|
||||
}
|
||||
|
||||
for (auto & [key, response] : exist_responses)
|
||||
{
|
||||
if (response.get().error == Coordination::Error::ZOK)
|
||||
{
|
||||
requests.push_back(zkutil::makeSetRequest(storage.fullPathForKey(*key), new_values[*key], -1));
|
||||
}
|
||||
else
|
||||
{
|
||||
requests.push_back(zkutil::makeCreateRequest(storage.fullPathForKey(*key), new_values[*key], zkutil::CreateMode::Persistent));
|
||||
++new_keys_num;
|
||||
}
|
||||
}
|
||||
|
||||
if (new_keys_num != 0)
|
||||
{
|
||||
auto will_be = current_keys_num + new_keys_num;
|
||||
if (keys_limit != 0 && will_be > keys_limit)
|
||||
throw Exception(
|
||||
ErrorCodes::LIMIT_EXCEEDED,
|
||||
"Limit would be exceeded by inserting {} new key(s). Limit is {}, while the number of keys would be {}",
|
||||
new_keys_num,
|
||||
keys_limit,
|
||||
will_be);
|
||||
}
|
||||
|
||||
zookeeper->multi(requests);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename KeyContainer>
|
||||
class StorageKeeperMapSource : public ISource
|
||||
{
|
||||
const StorageKeeperMap & storage;
|
||||
size_t max_block_size;
|
||||
|
||||
using KeyContainerPtr = std::shared_ptr<KeyContainer>;
|
||||
KeyContainerPtr container;
|
||||
using KeyContainerIter = typename KeyContainer::const_iterator;
|
||||
KeyContainerIter it;
|
||||
KeyContainerIter end;
|
||||
|
||||
public:
|
||||
StorageKeeperMapSource(
|
||||
const StorageKeeperMap & storage_,
|
||||
const Block & header,
|
||||
size_t max_block_size_,
|
||||
KeyContainerPtr container_,
|
||||
KeyContainerIter begin_,
|
||||
KeyContainerIter end_)
|
||||
: ISource(header), storage(storage_), max_block_size(max_block_size_), container(std::move(container_)), it(begin_), end(end_)
|
||||
{
|
||||
}
|
||||
|
||||
std::string getName() const override { return "StorageKeeperMapSource"; }
|
||||
|
||||
Chunk generate() override
|
||||
{
|
||||
if (it >= end)
|
||||
{
|
||||
it = {};
|
||||
return {};
|
||||
}
|
||||
|
||||
using KeyType = typename KeyContainer::value_type;
|
||||
if constexpr (std::same_as<KeyType, Field>)
|
||||
{
|
||||
const auto & sample_block = getPort().getHeader();
|
||||
const auto & key_column_type = sample_block.getByName(storage.getPrimaryKey().at(0)).type;
|
||||
auto raw_keys = serializeKeysToRawString(it, end, key_column_type, max_block_size);
|
||||
|
||||
for (auto & raw_key : raw_keys)
|
||||
raw_key = base64Encode(raw_key, /* url_encoding */ true);
|
||||
|
||||
return storage.getBySerializedKeys(raw_keys, nullptr);
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t elem_num = std::min(max_block_size, static_cast<size_t>(end - it));
|
||||
auto chunk = storage.getBySerializedKeys(std::span{it, it + elem_num}, nullptr);
|
||||
it += elem_num;
|
||||
return chunk;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
StorageKeeperMap::StorageKeeperMap(
|
||||
ContextPtr context_,
|
||||
const StorageID & table_id,
|
||||
const StorageInMemoryMetadata & metadata,
|
||||
bool attach,
|
||||
std::string_view primary_key_,
|
||||
const std::string & root_path_,
|
||||
UInt64 keys_limit_)
|
||||
: IStorage(table_id)
|
||||
, WithContext(context_->getGlobalContext())
|
||||
, root_path(zkutil::extractZooKeeperPath(root_path_, false))
|
||||
, primary_key(primary_key_)
|
||||
, zookeeper_name(zkutil::extractZooKeeperName(root_path_))
|
||||
, keys_limit(keys_limit_)
|
||||
, log(&Poco::Logger::get(fmt::format("StorageKeeperMap ({})", table_id.getNameForLogs())))
|
||||
{
|
||||
std::string path_prefix = context_->getConfigRef().getString("keeper_map_path_prefix", "");
|
||||
if (path_prefix.empty())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "KeeperMap is disabled because 'keeper_map_path_prefix' config is not defined");
|
||||
|
||||
verifyTableId(table_id);
|
||||
|
||||
setInMemoryMetadata(metadata);
|
||||
|
||||
WriteBufferFromOwnString out;
|
||||
out << "KeeperMap metadata format version: 1\n"
|
||||
<< "columns: " << metadata.columns.toString()
|
||||
<< "primary key: " << formattedAST(metadata.getPrimaryKey().expression_list_ast) << "\n";
|
||||
metadata_string = out.str();
|
||||
|
||||
if (root_path.empty())
|
||||
throw Exception("root_path should not be empty", ErrorCodes::BAD_ARGUMENTS);
|
||||
if (!root_path.starts_with('/'))
|
||||
throw Exception("root_path should start with '/'", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
auto config_keys_limit = context_->getConfigRef().getUInt64("keeper_map_keys_limit", 0);
|
||||
if (config_keys_limit != 0 && (keys_limit == 0 || keys_limit > config_keys_limit))
|
||||
{
|
||||
LOG_WARNING(
|
||||
log,
|
||||
"Keys limit defined by argument ({}) is larger than the one defined by 'keeper_map_keys_limit' config ({}). Will use "
|
||||
"config defined value",
|
||||
keys_limit,
|
||||
config_keys_limit);
|
||||
keys_limit = config_keys_limit;
|
||||
}
|
||||
else if (keys_limit > 0)
|
||||
{
|
||||
LOG_INFO(log, "Keys limit will be set to {}", keys_limit);
|
||||
}
|
||||
|
||||
auto root_path_fs = fs::path(path_prefix) / std::string_view{root_path}.substr(1);
|
||||
root_path = root_path_fs.generic_string();
|
||||
|
||||
data_path = root_path_fs / "data";
|
||||
|
||||
auto metadata_path_fs = root_path_fs / "metadata";
|
||||
metadata_path = metadata_path_fs;
|
||||
tables_path = metadata_path_fs / "tables";
|
||||
|
||||
auto table_unique_id = toString(table_id.uuid) + toString(ServerUUID::get());
|
||||
table_path = fs::path(tables_path) / table_unique_id;
|
||||
|
||||
dropped_path = metadata_path_fs / "dropped";
|
||||
dropped_lock_path = fs::path(dropped_path) / "lock";
|
||||
|
||||
if (attach)
|
||||
{
|
||||
checkTable<false>();
|
||||
return;
|
||||
}
|
||||
|
||||
auto client = getClient();
|
||||
|
||||
if (root_path != "/" && !client->exists(root_path))
|
||||
{
|
||||
LOG_TRACE(log, "Creating root path {}", root_path);
|
||||
client->createAncestors(root_path);
|
||||
client->createIfNotExists(root_path, "");
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < 1000; ++i)
|
||||
{
|
||||
if (client->exists(dropped_path))
|
||||
{
|
||||
LOG_INFO(log, "Removing leftover nodes");
|
||||
auto code = client->tryCreate(dropped_lock_path, "", zkutil::CreateMode::Ephemeral);
|
||||
|
||||
if (code == Coordination::Error::ZNONODE)
|
||||
{
|
||||
LOG_INFO(log, "Someone else removed leftover nodes");
|
||||
}
|
||||
else if (code == Coordination::Error::ZNODEEXISTS)
|
||||
{
|
||||
LOG_INFO(log, "Someone else is removing leftover nodes");
|
||||
continue;
|
||||
}
|
||||
else if (code != Coordination::Error::ZOK)
|
||||
{
|
||||
throw Coordination::Exception(code, dropped_lock_path);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(dropped_lock_path, *client);
|
||||
if (!dropTable(client, metadata_drop_lock))
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
std::string stored_metadata_string;
|
||||
auto exists = client->tryGet(metadata_path, stored_metadata_string);
|
||||
|
||||
if (exists)
|
||||
{
|
||||
// this requires same name for columns
|
||||
// maybe we can do a smarter comparison for columns and primary key expression
|
||||
if (stored_metadata_string != metadata_string)
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Path {} is already used but the stored table definition doesn't match. Stored metadata: {}",
|
||||
root_path,
|
||||
stored_metadata_string);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto code = client->tryCreate(metadata_path, metadata_string, zkutil::CreateMode::Persistent);
|
||||
if (code == Coordination::Error::ZNODEEXISTS)
|
||||
continue;
|
||||
else if (code != Coordination::Error::ZOK)
|
||||
throw Coordination::Exception(code, metadata_path);
|
||||
}
|
||||
|
||||
client->createIfNotExists(tables_path, "");
|
||||
|
||||
auto code = client->tryCreate(table_path, "", zkutil::CreateMode::Persistent);
|
||||
|
||||
if (code == Coordination::Error::ZOK)
|
||||
{
|
||||
// metadata now should be guaranteed to exist because we added our UUID to the tables_path
|
||||
client->createIfNotExists(data_path, "");
|
||||
table_is_valid = true;
|
||||
return;
|
||||
}
|
||||
|
||||
if (code == Coordination::Error::ZNONODE)
|
||||
LOG_INFO(log, "Metadata nodes were deleted in background, will retry");
|
||||
else
|
||||
throw Coordination::Exception(code, table_path);
|
||||
}
|
||||
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot create metadata for table, because it is removed concurrently or because of wrong root_path ({})", root_path);
|
||||
}
|
||||
|
||||
|
||||
Pipe StorageKeeperMap::read(
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
ContextPtr context_,
|
||||
QueryProcessingStage::Enum /*processed_stage*/,
|
||||
size_t max_block_size,
|
||||
unsigned num_streams)
|
||||
{
|
||||
checkTable<true>();
|
||||
storage_snapshot->check(column_names);
|
||||
|
||||
FieldVectorPtr filtered_keys;
|
||||
bool all_scan;
|
||||
|
||||
Block sample_block = storage_snapshot->metadata->getSampleBlock();
|
||||
auto primary_key_type = sample_block.getByName(primary_key).type;
|
||||
std::tie(filtered_keys, all_scan) = getFilterKeys(primary_key, primary_key_type, query_info, context_);
|
||||
|
||||
const auto process_keys = [&]<typename KeyContainerPtr>(KeyContainerPtr keys) -> Pipe
|
||||
{
|
||||
if (keys->empty())
|
||||
return {};
|
||||
|
||||
::sort(keys->begin(), keys->end());
|
||||
keys->erase(std::unique(keys->begin(), keys->end()), keys->end());
|
||||
|
||||
Pipes pipes;
|
||||
|
||||
size_t num_keys = keys->size();
|
||||
size_t num_threads = std::min<size_t>(num_streams, keys->size());
|
||||
|
||||
assert(num_keys <= std::numeric_limits<uint32_t>::max());
|
||||
assert(num_threads <= std::numeric_limits<uint32_t>::max());
|
||||
|
||||
for (size_t thread_idx = 0; thread_idx < num_threads; ++thread_idx)
|
||||
{
|
||||
size_t begin = num_keys * thread_idx / num_threads;
|
||||
size_t end = num_keys * (thread_idx + 1) / num_threads;
|
||||
|
||||
using KeyContainer = typename KeyContainerPtr::element_type;
|
||||
pipes.emplace_back(std::make_shared<StorageKeeperMapSource<KeyContainer>>(
|
||||
*this, sample_block, max_block_size, keys, keys->begin() + begin, keys->begin() + end));
|
||||
}
|
||||
return Pipe::unitePipes(std::move(pipes));
|
||||
};
|
||||
|
||||
auto client = getClient();
|
||||
if (all_scan)
|
||||
return process_keys(std::make_shared<std::vector<std::string>>(client->getChildren(data_path)));
|
||||
|
||||
return process_keys(std::move(filtered_keys));
|
||||
}
|
||||
|
||||
SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/)
|
||||
{
|
||||
checkTable<true>();
|
||||
return std::make_shared<StorageKeeperMapSink>(*this, metadata_snapshot);
|
||||
}
|
||||
|
||||
void StorageKeeperMap::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &)
|
||||
{
|
||||
checkTable<true>();
|
||||
auto client = getClient();
|
||||
client->tryRemoveChildrenRecursive(data_path, true);
|
||||
}
|
||||
|
||||
bool StorageKeeperMap::dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock)
|
||||
{
|
||||
zookeeper->removeChildrenRecursive(data_path);
|
||||
|
||||
bool completely_removed = false;
|
||||
Coordination::Requests ops;
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(metadata_drop_lock->getPath(), -1));
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(dropped_path, -1));
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(data_path, -1));
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(metadata_path, -1));
|
||||
|
||||
Coordination::Responses responses;
|
||||
auto code = zookeeper->tryMulti(ops, responses);
|
||||
using enum Coordination::Error;
|
||||
switch (code)
|
||||
{
|
||||
case ZOK:
|
||||
{
|
||||
metadata_drop_lock->setAlreadyRemoved();
|
||||
completely_removed = true;
|
||||
LOG_INFO(log, "Metadata ({}) and data ({}) was successfully removed from ZooKeeper", metadata_path, data_path);
|
||||
break;
|
||||
}
|
||||
case ZNONODE:
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "There is a race condition between creation and removal of metadata. It's a bug");
|
||||
case ZNOTEMPTY:
|
||||
LOG_ERROR(log, "Metadata was not completely removed from ZooKeeper");
|
||||
break;
|
||||
default:
|
||||
zkutil::KeeperMultiException::check(code, ops, responses);
|
||||
break;
|
||||
}
|
||||
return completely_removed;
|
||||
}
|
||||
|
||||
void StorageKeeperMap::drop()
|
||||
{
|
||||
checkTable<true>();
|
||||
auto client = getClient();
|
||||
|
||||
client->remove(table_path);
|
||||
|
||||
if (!client->getChildren(tables_path).empty())
|
||||
return;
|
||||
|
||||
Coordination::Requests ops;
|
||||
Coordination::Responses responses;
|
||||
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(tables_path, -1));
|
||||
ops.emplace_back(zkutil::makeCreateRequest(dropped_path, "", zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeCreateRequest(dropped_lock_path, "", zkutil::CreateMode::Ephemeral));
|
||||
|
||||
auto code = client->tryMulti(ops, responses);
|
||||
|
||||
if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS)
|
||||
{
|
||||
LOG_INFO(log, "Metadata is being removed by another table");
|
||||
return;
|
||||
}
|
||||
else if (code == Coordination::Error::ZNOTEMPTY)
|
||||
{
|
||||
LOG_WARNING(log, "Another table is using the same path, metadata will not be deleted");
|
||||
return;
|
||||
}
|
||||
else if (code != Coordination::Error::ZOK)
|
||||
zkutil::KeeperMultiException::check(code, ops, responses);
|
||||
|
||||
auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(dropped_lock_path, *client);
|
||||
dropTable(client, metadata_drop_lock);
|
||||
}
|
||||
|
||||
zkutil::ZooKeeperPtr StorageKeeperMap::getClient() const
|
||||
{
|
||||
std::lock_guard lock{zookeeper_mutex};
|
||||
if (!zookeeper_client || zookeeper_client->expired())
|
||||
{
|
||||
zookeeper_client = nullptr;
|
||||
if (zookeeper_name == "default")
|
||||
zookeeper_client = getContext()->getZooKeeper();
|
||||
else
|
||||
zookeeper_client = getContext()->getAuxiliaryZooKeeper(zookeeper_name);
|
||||
|
||||
zookeeper_client->sync(root_path);
|
||||
}
|
||||
|
||||
return zookeeper_client;
|
||||
}
|
||||
|
||||
const std::string & StorageKeeperMap::dataPath() const
|
||||
{
|
||||
return data_path;
|
||||
}
|
||||
|
||||
std::string StorageKeeperMap::fullPathForKey(const std::string_view key) const
|
||||
{
|
||||
return fs::path(data_path) / key;
|
||||
}
|
||||
|
||||
UInt64 StorageKeeperMap::keysLimit() const
|
||||
{
|
||||
return keys_limit;
|
||||
}
|
||||
|
||||
std::optional<bool> StorageKeeperMap::isTableValid() const
|
||||
{
|
||||
std::lock_guard lock{init_mutex};
|
||||
if (table_is_valid.has_value())
|
||||
return *table_is_valid;
|
||||
|
||||
[&]
|
||||
{
|
||||
try
|
||||
{
|
||||
auto client = getClient();
|
||||
|
||||
std::string stored_metadata_string;
|
||||
Coordination::Stat metadata_stat;
|
||||
client->tryGet(metadata_path, stored_metadata_string, &metadata_stat);
|
||||
|
||||
if (metadata_stat.numChildren == 0)
|
||||
{
|
||||
table_is_valid = false;
|
||||
return;
|
||||
}
|
||||
|
||||
if (metadata_string != stored_metadata_string)
|
||||
{
|
||||
LOG_ERROR(
|
||||
log,
|
||||
"Table definition does not match to the one stored in the path {}. Stored definition: {}",
|
||||
root_path,
|
||||
stored_metadata_string);
|
||||
table_is_valid = false;
|
||||
return;
|
||||
}
|
||||
|
||||
// validate all metadata and data nodes are present
|
||||
Coordination::Requests requests;
|
||||
requests.push_back(zkutil::makeCheckRequest(table_path, -1));
|
||||
requests.push_back(zkutil::makeCheckRequest(data_path, -1));
|
||||
requests.push_back(zkutil::makeCheckRequest(dropped_path, -1));
|
||||
|
||||
Coordination::Responses responses;
|
||||
client->tryMulti(requests, responses);
|
||||
|
||||
table_is_valid = false;
|
||||
if (responses[0]->error != Coordination::Error::ZOK)
|
||||
{
|
||||
LOG_ERROR(log, "Table node ({}) is missing", table_path);
|
||||
return;
|
||||
}
|
||||
|
||||
if (responses[1]->error != Coordination::Error::ZOK)
|
||||
{
|
||||
LOG_ERROR(log, "Data node ({}) is missing", data_path);
|
||||
return;
|
||||
}
|
||||
|
||||
if (responses[2]->error == Coordination::Error::ZOK)
|
||||
{
|
||||
LOG_ERROR(log, "Tables with root node {} are being dropped", root_path);
|
||||
return;
|
||||
}
|
||||
|
||||
table_is_valid = true;
|
||||
}
|
||||
catch (const Coordination::Exception & e)
|
||||
{
|
||||
tryLogCurrentException(log);
|
||||
|
||||
if (!Coordination::isHardwareError(e.code))
|
||||
table_is_valid = false;
|
||||
}
|
||||
}();
|
||||
|
||||
return table_is_valid;
|
||||
}
|
||||
|
||||
Chunk StorageKeeperMap::getByKeys(const ColumnsWithTypeAndName & keys, PaddedPODArray<UInt8> & null_map, const Names &) const
|
||||
{
|
||||
if (keys.size() != 1)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "StorageKeeperMap supports only one key, got: {}", keys.size());
|
||||
|
||||
auto raw_keys = serializeKeysToRawString(keys[0]);
|
||||
|
||||
if (raw_keys.size() != keys[0].column->size())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Assertion failed: {} != {}", raw_keys.size(), keys[0].column->size());
|
||||
|
||||
return getBySerializedKeys(raw_keys, &null_map);
|
||||
}
|
||||
|
||||
Chunk StorageKeeperMap::getBySerializedKeys(const std::span<const std::string> keys, PaddedPODArray<UInt8> * null_map) const
|
||||
{
|
||||
Block sample_block = getInMemoryMetadataPtr()->getSampleBlock();
|
||||
MutableColumns columns = sample_block.cloneEmptyColumns();
|
||||
size_t primary_key_pos = getPrimaryKeyPos(sample_block, getPrimaryKey());
|
||||
|
||||
if (null_map)
|
||||
{
|
||||
null_map->clear();
|
||||
null_map->resize_fill(keys.size(), 1);
|
||||
}
|
||||
|
||||
auto client = getClient();
|
||||
|
||||
std::vector<std::future<Coordination::GetResponse>> values;
|
||||
values.reserve(keys.size());
|
||||
|
||||
for (const auto & key : keys)
|
||||
{
|
||||
const auto full_path = fullPathForKey(key);
|
||||
values.emplace_back(client->asyncTryGet(full_path));
|
||||
}
|
||||
|
||||
auto wait_until = std::chrono::system_clock::now() + std::chrono::milliseconds(Coordination::DEFAULT_OPERATION_TIMEOUT_MS);
|
||||
|
||||
for (size_t i = 0; i < keys.size(); ++i)
|
||||
{
|
||||
auto & value = values[i];
|
||||
if (value.wait_until(wait_until) != std::future_status::ready)
|
||||
throw DB::Exception(ErrorCodes::KEEPER_EXCEPTION, "Failed to fetch values: timeout");
|
||||
|
||||
auto response = value.get();
|
||||
Coordination::Error code = response.error;
|
||||
|
||||
if (code == Coordination::Error::ZOK)
|
||||
{
|
||||
fillColumns(base64Decode(keys[i], true), response.data, primary_key_pos, sample_block, columns);
|
||||
}
|
||||
else if (code == Coordination::Error::ZNONODE)
|
||||
{
|
||||
if (null_map)
|
||||
{
|
||||
(*null_map)[i] = 0;
|
||||
for (size_t col_idx = 0; col_idx < sample_block.columns(); ++col_idx)
|
||||
columns[col_idx]->insert(sample_block.getByPosition(col_idx).type->getDefault());
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
throw DB::Exception(ErrorCodes::KEEPER_EXCEPTION, "Failed to fetch value: {}", code);
|
||||
}
|
||||
}
|
||||
|
||||
size_t num_rows = columns.at(0)->size();
|
||||
return Chunk(std::move(columns), num_rows);
|
||||
}
|
||||
|
||||
Block StorageKeeperMap::getSampleBlock(const Names &) const
|
||||
{
|
||||
auto metadata = getInMemoryMetadataPtr();
|
||||
return metadata->getSampleBlock();
|
||||
}
|
||||
|
||||
void StorageKeeperMap::checkTableCanBeRenamed(const StorageID & new_name) const
|
||||
{
|
||||
verifyTableId(new_name);
|
||||
}
|
||||
|
||||
void StorageKeeperMap::rename(const String & /*new_path_to_table_data*/, const StorageID & new_table_id)
|
||||
{
|
||||
checkTableCanBeRenamed(new_table_id);
|
||||
renameInMemory(new_table_id);
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
StoragePtr create(const StorageFactory::Arguments & args)
|
||||
{
|
||||
ASTs & engine_args = args.engine_args;
|
||||
if (engine_args.empty() || engine_args.size() > 2)
|
||||
throw Exception(
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Storage KeeperMap requires 1-3 arguments:\n"
|
||||
"root_path: path in the Keeper where the values will be stored (required)\n"
|
||||
"keys_limit: number of keys allowed to be stored, 0 is no limit (default: 0)");
|
||||
|
||||
const auto root_path_node = evaluateConstantExpressionAsLiteral(engine_args[0], args.getLocalContext());
|
||||
auto root_path = checkAndGetLiteralArgument<std::string>(root_path_node, "root_path");
|
||||
|
||||
UInt64 keys_limit = 0;
|
||||
if (engine_args.size() > 1)
|
||||
keys_limit = checkAndGetLiteralArgument<UInt64>(engine_args[1], "keys_limit");
|
||||
|
||||
StorageInMemoryMetadata metadata;
|
||||
metadata.setColumns(args.columns);
|
||||
metadata.setConstraints(args.constraints);
|
||||
|
||||
if (!args.storage_def->primary_key)
|
||||
throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.getContext());
|
||||
auto primary_key_names = metadata.getColumnsRequiredForPrimaryKey();
|
||||
if (primary_key_names.size() != 1)
|
||||
throw Exception("StorageKeeperMap requires one column in primary key", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
return std::make_shared<StorageKeeperMap>(
|
||||
args.getContext(), args.table_id, metadata, args.query.attach, primary_key_names[0], root_path, keys_limit);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void registerStorageKeeperMap(StorageFactory & factory)
|
||||
{
|
||||
factory.registerStorage(
|
||||
"KeeperMap",
|
||||
create,
|
||||
{
|
||||
.supports_sort_order = true,
|
||||
.supports_parallel_insert = true,
|
||||
});
|
||||
}
|
||||
|
||||
}
|
138
src/Storages/StorageKeeperMap.h
Normal file
138
src/Storages/StorageKeeperMap.h
Normal file
@ -0,0 +1,138 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/IKeyValueEntity.h>
|
||||
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
#include <Common/PODArray_fwd.h>
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
|
||||
#include <span>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INVALID_STATE;
|
||||
}
|
||||
|
||||
// KV store using (Zoo|CH)Keeper
|
||||
class StorageKeeperMap final : public IStorage, public IKeyValueEntity, WithContext
|
||||
{
|
||||
public:
|
||||
StorageKeeperMap(
|
||||
ContextPtr context_,
|
||||
const StorageID & table_id,
|
||||
const StorageInMemoryMetadata & metadata,
|
||||
bool attach,
|
||||
std::string_view primary_key_,
|
||||
const std::string & root_path_,
|
||||
UInt64 keys_limit_);
|
||||
|
||||
Pipe read(
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
ContextPtr context,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
size_t max_block_size,
|
||||
unsigned num_streams) override;
|
||||
|
||||
SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override;
|
||||
|
||||
void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override;
|
||||
void drop() override;
|
||||
|
||||
std::string getName() const override { return "KeeperMap"; }
|
||||
Names getPrimaryKey() const override { return {primary_key}; }
|
||||
|
||||
Chunk getByKeys(const ColumnsWithTypeAndName & keys, PaddedPODArray<UInt8> & null_map, const Names &) const override;
|
||||
Chunk getBySerializedKeys(std::span<const std::string> keys, PaddedPODArray<UInt8> * null_map) const;
|
||||
|
||||
Block getSampleBlock(const Names &) const override;
|
||||
|
||||
void checkTableCanBeRenamed(const StorageID & new_name) const override;
|
||||
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
|
||||
|
||||
bool supportsParallelInsert() const override { return true; }
|
||||
bool supportsIndexForIn() const override { return true; }
|
||||
bool mayBenefitFromIndexForIn(
|
||||
const ASTPtr & node, ContextPtr /*query_context*/, const StorageMetadataPtr & /*metadata_snapshot*/) const override
|
||||
{
|
||||
return node->getColumnName() == primary_key;
|
||||
}
|
||||
|
||||
zkutil::ZooKeeperPtr getClient() const;
|
||||
const std::string & dataPath() const;
|
||||
std::string fullPathForKey(std::string_view key) const;
|
||||
|
||||
UInt64 keysLimit() const;
|
||||
|
||||
template <bool throw_on_error>
|
||||
void checkTable() const
|
||||
{
|
||||
auto is_table_valid = isTableValid();
|
||||
if (!is_table_valid.has_value())
|
||||
{
|
||||
static constexpr std::string_view error_msg = "Failed to activate table because of connection issues. It will be activated "
|
||||
"once a connection is established and metadata is verified";
|
||||
if constexpr (throw_on_error)
|
||||
throw Exception(ErrorCodes::INVALID_STATE, error_msg);
|
||||
else
|
||||
{
|
||||
LOG_ERROR(log, fmt::runtime(error_msg));
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
if (!*is_table_valid)
|
||||
{
|
||||
static constexpr std::string_view error_msg
|
||||
= "Failed to activate table because of invalid metadata in ZooKeeper. Please DETACH table";
|
||||
if constexpr (throw_on_error)
|
||||
throw Exception(ErrorCodes::INVALID_STATE, error_msg);
|
||||
else
|
||||
{
|
||||
LOG_ERROR(log, fmt::runtime(error_msg));
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
bool dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock);
|
||||
|
||||
std::optional<bool> isTableValid() const;
|
||||
|
||||
std::string root_path;
|
||||
std::string primary_key;
|
||||
|
||||
std::string data_path;
|
||||
|
||||
std::string metadata_path;
|
||||
|
||||
std::string tables_path;
|
||||
std::string table_path;
|
||||
|
||||
std::string dropped_path;
|
||||
std::string dropped_lock_path;
|
||||
|
||||
std::string zookeeper_name;
|
||||
|
||||
std::string metadata_string;
|
||||
|
||||
uint64_t keys_limit{0};
|
||||
|
||||
mutable std::mutex zookeeper_mutex;
|
||||
mutable zkutil::ZooKeeperPtr zookeeper_client{nullptr};
|
||||
|
||||
mutable std::mutex init_mutex;
|
||||
mutable std::optional<bool> table_is_valid;
|
||||
|
||||
Poco::Logger * log;
|
||||
};
|
||||
|
||||
}
|
@ -8222,25 +8222,12 @@ void StorageReplicatedMergeTree::createZeroCopyLockNode(
|
||||
}
|
||||
}
|
||||
|
||||
bool StorageReplicatedMergeTree::removeDetachedPart(DiskPtr disk, const String & path, const String & part_name, bool is_freezed)
|
||||
bool StorageReplicatedMergeTree::removeDetachedPart(DiskPtr disk, const String & path, const String & part_name)
|
||||
{
|
||||
if (disk->supportZeroCopyReplication())
|
||||
{
|
||||
if (is_freezed)
|
||||
{
|
||||
FreezeMetaData meta;
|
||||
if (meta.load(disk, path))
|
||||
{
|
||||
FreezeMetaData::clean(disk, path);
|
||||
return removeSharedDetachedPart(disk, path, part_name, meta.table_shared_id, meta.zookeeper_name, meta.replica_name, "", getContext());
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
String table_id = getTableSharedID();
|
||||
|
||||
return removeSharedDetachedPart(disk, path, part_name, table_id, zookeeper_name, replica_name, zookeeper_path, getContext());
|
||||
}
|
||||
String table_id = getTableSharedID();
|
||||
return removeSharedDetachedPart(disk, path, part_name, table_id, zookeeper_name, replica_name, zookeeper_path, getContext(), current_zookeeper);
|
||||
}
|
||||
|
||||
disk->removeRecursive(path);
|
||||
@ -8250,11 +8237,10 @@ bool StorageReplicatedMergeTree::removeDetachedPart(DiskPtr disk, const String &
|
||||
|
||||
|
||||
bool StorageReplicatedMergeTree::removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, const String & table_uuid,
|
||||
const String &, const String & detached_replica_name, const String & detached_zookeeper_path, ContextPtr local_context)
|
||||
const String &, const String & detached_replica_name, const String & detached_zookeeper_path, ContextPtr local_context, const zkutil::ZooKeeperPtr & zookeeper)
|
||||
{
|
||||
bool keep_shared = false;
|
||||
|
||||
zkutil::ZooKeeperPtr zookeeper = local_context->getZooKeeper();
|
||||
NameSet files_not_to_remove;
|
||||
|
||||
fs::path checksums = fs::path(path) / IMergeTreeDataPart::FILE_FOR_REFERENCES_CHECK;
|
||||
|
@ -325,7 +325,7 @@ public:
|
||||
void checkBrokenDisks();
|
||||
|
||||
static bool removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, const String & table_uuid,
|
||||
const String & zookeeper_name, const String & replica_name, const String & zookeeper_path, ContextPtr local_context);
|
||||
const String & zookeeper_name, const String & replica_name, const String & zookeeper_path, ContextPtr local_context, const zkutil::ZooKeeperPtr & zookeeper);
|
||||
|
||||
bool canUseZeroCopyReplication() const;
|
||||
private:
|
||||
@ -834,7 +834,7 @@ private:
|
||||
int32_t mode = zkutil::CreateMode::Persistent, bool replace_existing_lock = false,
|
||||
const String & path_to_set_hardlinked_files = "", const NameSet & hardlinked_files = {});
|
||||
|
||||
bool removeDetachedPart(DiskPtr disk, const String & path, const String & part_name, bool is_freezed) override;
|
||||
bool removeDetachedPart(DiskPtr disk, const String & path, const String & part_name) override;
|
||||
|
||||
/// Create freeze metadata for table and save in zookeeper. Required only if zero-copy replication enabled.
|
||||
void createAndStoreFreezeMetadata(DiskPtr disk, DataPartPtr part, String backup_part_path) const override;
|
||||
|
@ -88,6 +88,7 @@ void registerStorageFileLog(StorageFactory & factory);
|
||||
void registerStorageSQLite(StorageFactory & factory);
|
||||
#endif
|
||||
|
||||
void registerStorageKeeperMap(StorageFactory & factory);
|
||||
|
||||
void registerStorages()
|
||||
{
|
||||
@ -171,6 +172,8 @@ void registerStorages()
|
||||
#if USE_SQLITE
|
||||
registerStorageSQLite(factory);
|
||||
#endif
|
||||
|
||||
registerStorageKeeperMap(factory);
|
||||
}
|
||||
|
||||
}
|
||||
|
3
tests/config/config.d/enable_keeper_map.xml
Normal file
3
tests/config/config.d/enable_keeper_map.xml
Normal file
@ -0,0 +1,3 @@
|
||||
<clickhouse>
|
||||
<keeper_map_path_prefix>/test_keeper_map</keeper_map_path_prefix>
|
||||
</clickhouse>
|
4
tests/config/config.d/system_unfreeze.xml
Normal file
4
tests/config/config.d/system_unfreeze.xml
Normal file
@ -0,0 +1,4 @@
|
||||
<?xml version="1.0"?>
|
||||
<clickhouse>
|
||||
<enable_system_unfreeze>true</enable_system_unfreeze>
|
||||
</clickhouse>
|
@ -47,8 +47,10 @@ ln -sf $SRC_PATH/config.d/named_collection.xml $DEST_SERVER_PATH/config.d/
|
||||
ln -sf $SRC_PATH/config.d/ssl_certs.xml $DEST_SERVER_PATH/config.d/
|
||||
ln -sf $SRC_PATH/config.d/filesystem_cache_log.xml $DEST_SERVER_PATH/config.d/
|
||||
ln -sf $SRC_PATH/config.d/session_log.xml $DEST_SERVER_PATH/config.d/
|
||||
ln -sf $SRC_PATH/config.d/system_unfreeze.xml $DEST_SERVER_PATH/config.d/
|
||||
ln -sf $SRC_PATH/config.d/enable_zero_copy_replication.xml $DEST_SERVER_PATH/config.d/
|
||||
ln -sf $SRC_PATH/config.d/nlp.xml $DEST_SERVER_PATH/config.d/
|
||||
ln -sf $SRC_PATH/config.d/enable_keeper_map.xml $DEST_SERVER_PATH/config.d/
|
||||
|
||||
ln -sf $SRC_PATH/users.d/log_queries.xml $DEST_SERVER_PATH/users.d/
|
||||
ln -sf $SRC_PATH/users.d/readonly.xml $DEST_SERVER_PATH/users.d/
|
||||
@ -61,6 +63,7 @@ ln -sf $SRC_PATH/users.d/memory_profiler.xml $DEST_SERVER_PATH/users.d/
|
||||
ln -sf $SRC_PATH/users.d/no_fsync_metadata.xml $DEST_SERVER_PATH/users.d/
|
||||
ln -sf $SRC_PATH/users.d/filelog.xml $DEST_SERVER_PATH/users.d/
|
||||
ln -sf $SRC_PATH/users.d/enable_blobs_check.xml $DEST_SERVER_PATH/users.d/
|
||||
ln -sf $SRC_PATH/users.d/marks.xml $DEST_SERVER_PATH/users.d/
|
||||
|
||||
# FIXME DataPartsExchange may hang for http_send_timeout seconds
|
||||
# when nobody is going to read from the other side of socket (due to "Fetching of part was cancelled"),
|
||||
|
7
tests/config/users.d/marks.xml
Normal file
7
tests/config/users.d/marks.xml
Normal file
@ -0,0 +1,7 @@
|
||||
<clickhouse>
|
||||
<profiles>
|
||||
<default>
|
||||
<load_marks_asynchronously>1</load_marks_asynchronously>
|
||||
</default>
|
||||
</profiles>
|
||||
</clickhouse>
|
1
tests/integration/test_keeper_map/__init__.py
Normal file
1
tests/integration/test_keeper_map/__init__.py
Normal file
@ -0,0 +1 @@
|
||||
#!/usr/bin/env python3
|
@ -0,0 +1,3 @@
|
||||
<clickhouse>
|
||||
<keeper_map_path_prefix>/test_keeper_map</keeper_map_path_prefix>
|
||||
</clickhouse>
|
179
tests/integration/test_keeper_map/test.py
Normal file
179
tests/integration/test_keeper_map/test.py
Normal file
@ -0,0 +1,179 @@
|
||||
import multiprocessing
|
||||
import pytest
|
||||
from time import sleep
|
||||
import random
|
||||
from itertools import count
|
||||
from sys import stdout
|
||||
|
||||
from multiprocessing import Pool
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.test_tools import assert_eq_with_retry, assert_logs_contain
|
||||
from helpers.network import PartitionManager
|
||||
|
||||
test_recover_staled_replica_run = 1
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
node = cluster.add_instance(
|
||||
"node",
|
||||
main_configs=["configs/enable_keeper_map.xml"],
|
||||
with_zookeeper=True,
|
||||
stay_alive=True,
|
||||
)
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
yield cluster
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
def get_genuine_zk():
|
||||
return cluster.get_kazoo_client("zoo1")
|
||||
|
||||
|
||||
def remove_children(client, path):
|
||||
children = client.get_children(path)
|
||||
|
||||
for child in children:
|
||||
child_path = f"{path}/{child}"
|
||||
remove_children(client, child_path)
|
||||
client.delete(child_path)
|
||||
|
||||
|
||||
def test_create_keeper_map(started_cluster):
|
||||
node.query(
|
||||
"CREATE TABLE test_keeper_map (key UInt64, value UInt64) ENGINE = KeeperMap('/test1') PRIMARY KEY(key);"
|
||||
)
|
||||
zk_client = get_genuine_zk()
|
||||
|
||||
def assert_children_size(path, expected_size):
|
||||
assert len(zk_client.get_children(path)) == expected_size
|
||||
|
||||
def assert_root_children_size(expected_size):
|
||||
assert_children_size("/test_keeper_map/test1", expected_size)
|
||||
|
||||
def assert_data_children_size(expected_size):
|
||||
assert_children_size("/test_keeper_map/test1/data", expected_size)
|
||||
|
||||
assert_root_children_size(2)
|
||||
assert_data_children_size(0)
|
||||
|
||||
node.query("INSERT INTO test_keeper_map VALUES (1, 11)")
|
||||
assert_data_children_size(1)
|
||||
|
||||
node.query(
|
||||
"CREATE TABLE test_keeper_map_another (key UInt64, value UInt64) ENGINE = KeeperMap('/test1') PRIMARY KEY(key);"
|
||||
)
|
||||
assert_root_children_size(2)
|
||||
assert_data_children_size(1)
|
||||
|
||||
node.query("INSERT INTO test_keeper_map_another VALUES (1, 11)")
|
||||
assert_root_children_size(2)
|
||||
assert_data_children_size(1)
|
||||
|
||||
node.query("INSERT INTO test_keeper_map_another VALUES (2, 22)")
|
||||
assert_root_children_size(2)
|
||||
assert_data_children_size(2)
|
||||
|
||||
node.query("DROP TABLE test_keeper_map SYNC")
|
||||
assert_root_children_size(2)
|
||||
assert_data_children_size(2)
|
||||
|
||||
node.query("DROP TABLE test_keeper_map_another SYNC")
|
||||
assert_root_children_size(0)
|
||||
|
||||
zk_client.stop()
|
||||
|
||||
|
||||
def create_drop_loop(index, stop_event):
|
||||
table_name = f"test_keeper_map_{index}"
|
||||
|
||||
for i in count(0, 1):
|
||||
if stop_event.is_set():
|
||||
return
|
||||
|
||||
node.query(
|
||||
f"CREATE TABLE {table_name} (key UInt64, value UInt64) ENGINE = KeeperMap('/test') PRIMARY KEY(key);"
|
||||
)
|
||||
node.query(f"INSERT INTO {table_name} VALUES ({index}, {i})")
|
||||
result = node.query(f"SELECT value FROM {table_name} WHERE key = {index}")
|
||||
assert result.strip() == str(i)
|
||||
node.query(f"DROP TABLE {table_name} SYNC")
|
||||
|
||||
|
||||
def test_create_drop_keeper_map_concurrent(started_cluster):
|
||||
pool = Pool()
|
||||
manager = multiprocessing.Manager()
|
||||
stop_event = manager.Event()
|
||||
results = []
|
||||
for i in range(multiprocessing.cpu_count()):
|
||||
sleep(0.2)
|
||||
results.append(
|
||||
pool.apply_async(
|
||||
create_drop_loop,
|
||||
args=(
|
||||
i,
|
||||
stop_event,
|
||||
),
|
||||
)
|
||||
)
|
||||
|
||||
sleep(60)
|
||||
stop_event.set()
|
||||
|
||||
for result in results:
|
||||
result.get()
|
||||
|
||||
pool.close()
|
||||
|
||||
client = get_genuine_zk()
|
||||
assert len(client.get_children("/test_keeper_map/test")) == 0
|
||||
client.stop()
|
||||
|
||||
|
||||
def test_keeper_map_without_zk(started_cluster):
|
||||
def assert_keeper_exception_after_partition(query):
|
||||
with PartitionManager() as pm:
|
||||
pm.drop_instance_zk_connections(node)
|
||||
error = node.query_and_get_error(query)
|
||||
assert "Coordination::Exception" in error
|
||||
|
||||
assert_keeper_exception_after_partition(
|
||||
"CREATE TABLE test_keeper_map (key UInt64, value UInt64) ENGINE = KeeperMap('/test1') PRIMARY KEY(key);"
|
||||
)
|
||||
|
||||
node.query(
|
||||
"CREATE TABLE test_keeper_map (key UInt64, value UInt64) ENGINE = KeeperMap('/test1') PRIMARY KEY(key);"
|
||||
)
|
||||
|
||||
assert_keeper_exception_after_partition(
|
||||
"INSERT INTO test_keeper_map VALUES (1, 11)"
|
||||
)
|
||||
node.query("INSERT INTO test_keeper_map VALUES (1, 11)")
|
||||
|
||||
assert_keeper_exception_after_partition("SELECT * FROM test_keeper_map")
|
||||
node.query("SELECT * FROM test_keeper_map")
|
||||
|
||||
with PartitionManager() as pm:
|
||||
pm.drop_instance_zk_connections(node)
|
||||
node.restart_clickhouse(60)
|
||||
error = node.query_and_get_error("SELECT * FROM test_keeper_map")
|
||||
assert "Failed to activate table because of connection issues" in error
|
||||
|
||||
node.query("SELECT * FROM test_keeper_map")
|
||||
|
||||
client = get_genuine_zk()
|
||||
remove_children(client, "/test_keeper_map/test1")
|
||||
node.restart_clickhouse(60)
|
||||
error = node.query_and_get_error("SELECT * FROM test_keeper_map")
|
||||
assert "Failed to activate table because of invalid metadata in ZooKeeper" in error
|
||||
|
||||
node.query("DETACH TABLE test_keeper_map")
|
||||
|
||||
client.stop()
|
@ -1,18 +1,3 @@
|
||||
<clickhouse>
|
||||
<tcp_port>9000</tcp_port>
|
||||
<listen_host>127.0.0.1</listen_host>
|
||||
|
||||
<openSSL>
|
||||
<client>
|
||||
<cacheSessions>true</cacheSessions>
|
||||
<verificationMode>none</verificationMode>
|
||||
<invalidCertificateHandler>
|
||||
<name>AcceptCertificateHandler</name>
|
||||
</invalidCertificateHandler>
|
||||
</client>
|
||||
</openSSL>
|
||||
|
||||
<max_concurrent_queries>500</max_concurrent_queries>
|
||||
<path>./clickhouse/</path>
|
||||
<users_config>users.xml</users_config>
|
||||
<enable_system_unfreeze>true</enable_system_unfreeze>
|
||||
</clickhouse>
|
||||
|
@ -17,6 +17,7 @@ def cluster():
|
||||
cluster.add_instance(
|
||||
"node",
|
||||
main_configs=[
|
||||
"configs/config.xml",
|
||||
"configs/config.d/storage_conf.xml",
|
||||
"configs/config.d/bg_processing_pool_conf.xml",
|
||||
],
|
||||
@ -541,6 +542,8 @@ def test_freeze_unfreeze(cluster, node_name):
|
||||
# Unfreeze all partitions from backup2.
|
||||
node.query("ALTER TABLE s3_test UNFREEZE WITH NAME 'backup2'")
|
||||
|
||||
wait_for_delete_s3_objects(cluster, FILES_OVERHEAD)
|
||||
|
||||
# Data should be removed from S3.
|
||||
assert (
|
||||
len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)))
|
||||
@ -574,6 +577,8 @@ def test_freeze_system_unfreeze(cluster, node_name):
|
||||
# Unfreeze all data from backup3.
|
||||
node.query("SYSTEM UNFREEZE WITH NAME 'backup3'")
|
||||
|
||||
wait_for_delete_s3_objects(cluster, FILES_OVERHEAD)
|
||||
|
||||
# Data should be removed from S3.
|
||||
assert (
|
||||
len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)))
|
||||
|
@ -1,5 +1,5 @@
|
||||
<clickhouse>
|
||||
|
||||
<enable_system_unfreeze>true</enable_system_unfreeze>
|
||||
<storage_configuration>
|
||||
<disks>
|
||||
<s31>
|
||||
|
@ -6,4 +6,5 @@
|
||||
<mysql_port from_zk="/clickhouse/ports/mysql" replace="replace" />
|
||||
<postgresql_port from_zk="/clickhouse/ports/postgresql" replace="replace" />
|
||||
<grpc_port from_zk="/clickhouse/ports/grpc" replace="replace" />
|
||||
</yandex>
|
||||
<http_handlers from_zk="/clickhouse/http_handlers" replace="replace" />
|
||||
</yandex>
|
@ -25,7 +25,7 @@ cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance(
|
||||
"instance",
|
||||
main_configs=[
|
||||
"configs/ports_from_zk.xml",
|
||||
"configs/overrides_from_zk.xml",
|
||||
"configs/ssl_conf.xml",
|
||||
"configs/dhparam.pem",
|
||||
"configs/server.crt",
|
||||
@ -58,7 +58,7 @@ import clickhouse_grpc_pb2_grpc
|
||||
@pytest.fixture(name="cluster", scope="module")
|
||||
def fixture_cluster():
|
||||
try:
|
||||
cluster.add_zookeeper_startup_command(configure_ports_from_zk)
|
||||
cluster.add_zookeeper_startup_command(configure_from_zk)
|
||||
cluster.start()
|
||||
yield cluster
|
||||
finally:
|
||||
@ -128,7 +128,7 @@ def grpc_query(channel, query_text):
|
||||
return result.output.decode()
|
||||
|
||||
|
||||
def configure_ports_from_zk(zk, querier=None):
|
||||
def configure_from_zk(zk, querier=None):
|
||||
default_config = [
|
||||
("/clickhouse/listen_hosts", b"<listen_host>0.0.0.0</listen_host>"),
|
||||
("/clickhouse/ports/tcp", b"9000"),
|
||||
@ -136,6 +136,7 @@ def configure_ports_from_zk(zk, querier=None):
|
||||
("/clickhouse/ports/mysql", b"9004"),
|
||||
("/clickhouse/ports/postgresql", b"9005"),
|
||||
("/clickhouse/ports/grpc", b"9100"),
|
||||
("/clickhouse/http_handlers", b"<defaults/>"),
|
||||
]
|
||||
for path, value in default_config:
|
||||
if querier is not None:
|
||||
@ -182,7 +183,7 @@ def default_client(cluster, zk, restore_via_http=False):
|
||||
yield client
|
||||
finally:
|
||||
querier = instance.http_query if restore_via_http else client.query
|
||||
configure_ports_from_zk(zk, querier)
|
||||
configure_from_zk(zk, querier)
|
||||
|
||||
|
||||
def test_change_tcp_port(cluster, zk):
|
||||
@ -320,7 +321,7 @@ def test_change_listen_host(cluster, zk):
|
||||
assert localhost_client.query("SELECT 1") == "1\n"
|
||||
finally:
|
||||
with sync_loaded_config(localhost_client.query):
|
||||
configure_ports_from_zk(zk)
|
||||
configure_from_zk(zk)
|
||||
|
||||
|
||||
# This is a regression test for the case when the clickhouse-server was waiting
|
||||
@ -371,7 +372,7 @@ def test_reload_via_client(cluster, zk):
|
||||
while True:
|
||||
try:
|
||||
with sync_loaded_config(localhost_client.query):
|
||||
configure_ports_from_zk(zk)
|
||||
configure_from_zk(zk)
|
||||
break
|
||||
except QueryRuntimeException:
|
||||
logging.exception("The new socket is not binded yet")
|
||||
@ -379,3 +380,33 @@ def test_reload_via_client(cluster, zk):
|
||||
|
||||
if exception:
|
||||
raise exception
|
||||
|
||||
|
||||
def test_change_http_handlers(cluster, zk):
|
||||
with default_client(cluster, zk) as client:
|
||||
curl_result = instance.exec_in_container(
|
||||
["bash", "-c", "curl -s '127.0.0.1:8123/it_works'"]
|
||||
)
|
||||
assert "There is no handle /it_works" in curl_result
|
||||
|
||||
with sync_loaded_config(client.query):
|
||||
zk.set(
|
||||
"/clickhouse/http_handlers",
|
||||
b"""
|
||||
<defaults/>
|
||||
|
||||
<rule>
|
||||
<url>/it_works</url>
|
||||
<methods>GET</methods>
|
||||
<handler>
|
||||
<type>predefined_query_handler</type>
|
||||
<query>SELECT 'It works.'</query>
|
||||
</handler>
|
||||
</rule>
|
||||
""",
|
||||
)
|
||||
|
||||
curl_result = instance.exec_in_container(
|
||||
["bash", "-c", "curl -s '127.0.0.1:8123/it_works'"]
|
||||
)
|
||||
assert curl_result == "It works.\n"
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
SET prefer_localhost_replica = 1;
|
||||
|
||||
SELECT count() FROM remote('127.0.0.1,localhos', system.one); -- { serverError 279 }
|
||||
SELECT count() FROM remote('127.0.0.1,localhos', system.one); -- { serverError 198 }
|
||||
SELECT count() FROM remote('127.0.0.1|localhos', system.one);
|
||||
|
||||
-- Clear cache to avoid future errors in the logs
|
||||
|
@ -5,7 +5,7 @@ SELECT count() > 0 FROM system.stack_trace WHERE query_id != '';
|
||||
SELECT countIf(thread_id > 0) > 0 FROM system.stack_trace;
|
||||
1
|
||||
-- optimization for trace
|
||||
SELECT length(trace) > 0 FROM system.stack_trace LIMIT 1;
|
||||
SELECT count(trace) > 0 FROM system.stack_trace WHERE length(trace) > 0 LIMIT 1;
|
||||
1
|
||||
-- optimization for query_id
|
||||
SELECT length(query_id) > 0 FROM system.stack_trace WHERE query_id != '' LIMIT 1;
|
||||
|
@ -5,7 +5,7 @@ SELECT count() > 0 FROM system.stack_trace WHERE query_id != '';
|
||||
-- opimization for not reading /proc/self/task/{}/comm and avoid sending signal
|
||||
SELECT countIf(thread_id > 0) > 0 FROM system.stack_trace;
|
||||
-- optimization for trace
|
||||
SELECT length(trace) > 0 FROM system.stack_trace LIMIT 1;
|
||||
SELECT count(trace) > 0 FROM system.stack_trace WHERE length(trace) > 0 LIMIT 1;
|
||||
-- optimization for query_id
|
||||
SELECT length(query_id) > 0 FROM system.stack_trace WHERE query_id != '' LIMIT 1;
|
||||
-- optimization for thread_name
|
||||
|
@ -2,6 +2,7 @@ DROP TABLE IF EXISTS data_01283;
|
||||
|
||||
set remote_filesystem_read_method = 'read';
|
||||
set local_filesystem_read_method = 'pread';
|
||||
set load_marks_asynchronously = 0;
|
||||
|
||||
CREATE TABLE data_01283 engine=MergeTree()
|
||||
ORDER BY key
|
||||
|
@ -2,6 +2,7 @@ drop table if exists table_01323_many_parts;
|
||||
|
||||
set remote_filesystem_read_method = 'read';
|
||||
set local_filesystem_read_method = 'pread';
|
||||
set load_marks_asynchronously = 0;
|
||||
|
||||
create table table_01323_many_parts (x UInt64) engine = MergeTree order by x partition by x % 100;
|
||||
set max_partitions_per_insert_block = 100;
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-replicated-database, no-parallel, no-ordinary-database
|
||||
# Tags: no-replicated-database, no-parallel
|
||||
# Tag no-replicated-database: Unsupported type of ALTER query
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
|
@ -33,6 +33,7 @@ OPTIMIZE TABLE select_final FINAL;
|
||||
|
||||
SET remote_filesystem_read_method = 'read';
|
||||
SET local_filesystem_read_method = 'pread';
|
||||
set load_marks_asynchronously = 0;
|
||||
|
||||
SELECT max(x) FROM select_final FINAL;
|
||||
|
||||
|
@ -30,7 +30,7 @@ EOF
|
||||
${CLICKHOUSE_CLIENT} -q "SYSTEM STOP MERGES lazy_mark_test"
|
||||
${CLICKHOUSE_CLIENT} -q "INSERT INTO lazy_mark_test select number, number % 3, number % 5, number % 10, number % 13, number % 15, number % 17, number % 18, number % 22, number % 25 from numbers(1000000)"
|
||||
${CLICKHOUSE_CLIENT} -q "SYSTEM DROP MARK CACHE"
|
||||
${CLICKHOUSE_CLIENT} --log_queries=1 --query_id "${QUERY_ID}" -q "SELECT * FROM lazy_mark_test WHERE n3==11"
|
||||
${CLICKHOUSE_CLIENT} --log_queries=1 --query_id "${QUERY_ID}" -q "SELECT * FROM lazy_mark_test WHERE n3==11 SETTINGS load_marks_asynchronously=0"
|
||||
${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "select ProfileEvents['FileOpen'] from system.query_log where query_id = '${QUERY_ID}' and type = 'QueryFinish' and current_database = currentDatabase()"
|
||||
|
6
tests/queries/0_stateless/02416_keeper_map.reference
Normal file
6
tests/queries/0_stateless/02416_keeper_map.reference
Normal file
@ -0,0 +1,6 @@
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1 1 1 1 1
|
||||
1
|
38
tests/queries/0_stateless/02416_keeper_map.sql
Normal file
38
tests/queries/0_stateless/02416_keeper_map.sql
Normal file
@ -0,0 +1,38 @@
|
||||
-- Tags: no-ordinary-database, no-fasttest, long
|
||||
|
||||
DROP TABLE IF EXISTS 02416_test SYNC;
|
||||
|
||||
CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416'); -- { serverError 36 }
|
||||
CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(key2); -- { serverError 47 }
|
||||
CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(key, value); -- { serverError 36 }
|
||||
CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(concat(key, value)); -- { serverError 36 }
|
||||
CREATE TABLE 02416_test (key Tuple(String, UInt32), value UInt64) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(key);
|
||||
|
||||
DROP TABLE IF EXISTS 02416_test SYNC;
|
||||
CREATE TABLE 02416_test (key String, value UInt32) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(key);
|
||||
|
||||
INSERT INTO 02416_test SELECT '1_1', number FROM numbers(1000);
|
||||
SELECT COUNT(1) == 1 FROM 02416_test;
|
||||
|
||||
INSERT INTO 02416_test SELECT concat(toString(number), '_1'), number FROM numbers(1000);
|
||||
SELECT COUNT(1) == 1000 FROM 02416_test;
|
||||
SELECT uniqExact(key) == 32 FROM (SELECT * FROM 02416_test LIMIT 32 SETTINGS max_block_size = 1);
|
||||
SELECT SUM(value) == 1 + 99 + 900 FROM 02416_test WHERE key IN ('1_1', '99_1', '900_1');
|
||||
|
||||
DROP TABLE IF EXISTS 02416_test SYNC;
|
||||
DROP TABLE IF EXISTS 02416_test_memory;
|
||||
|
||||
CREATE TABLE 02416_test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=KeeperMap('/' || currentDatabase() || '/test2416') PRIMARY KEY(k);
|
||||
CREATE TABLE 02416_test_memory AS 02416_test Engine = Memory;
|
||||
|
||||
INSERT INTO 02416_test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000) group by k;
|
||||
|
||||
INSERT INTO 02416_test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000) group by k;
|
||||
|
||||
SELECT A.a = B.a, A.b = B.b, A.c = B.c, A.d = B.d, A.e = B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 02416_test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 02416_test_memory) B USING a ORDER BY a;
|
||||
|
||||
TRUNCATE TABLE 02416_test;
|
||||
SELECT 0 == COUNT(1) FROM 02416_test;
|
||||
|
||||
DROP TABLE IF EXISTS 02416_test SYNC;
|
||||
DROP TABLE IF EXISTS 02416_test_memory;
|
@ -0,0 +1,10 @@
|
||||
1 11
|
||||
------
|
||||
1 11
|
||||
2 22
|
||||
------
|
||||
1 11
|
||||
2 22
|
||||
------
|
||||
1 11
|
||||
2 22
|
20
tests/queries/0_stateless/02417_keeper_map_create_drop.sql
Normal file
20
tests/queries/0_stateless/02417_keeper_map_create_drop.sql
Normal file
@ -0,0 +1,20 @@
|
||||
-- Tags: no-ordinary-database, no-fasttest
|
||||
|
||||
DROP TABLE IF EXISTS 02417_test SYNC;
|
||||
|
||||
CREATE TABLE 02417_test (key UInt64, value UInt64) Engine=KeeperMap('/' || currentDatabase() || '/test2417') PRIMARY KEY(key);
|
||||
INSERT INTO 02417_test VALUES (1, 11);
|
||||
SELECT * FROM 02417_test ORDER BY key;
|
||||
SELECT '------';
|
||||
|
||||
CREATE TABLE 02417_test_another (key UInt64, value UInt64) Engine=KeeperMap('/' || currentDatabase() || '/test2417') PRIMARY KEY(key);
|
||||
INSERT INTO 02417_test_another VALUES (2, 22);
|
||||
SELECT * FROM 02417_test_another ORDER BY key;
|
||||
SELECT '------';
|
||||
SELECT * FROM 02417_test ORDER BY key;
|
||||
SELECT '------';
|
||||
|
||||
DROP TABLE 02417_test SYNC;
|
||||
SELECT * FROM 02417_test_another ORDER BY key;
|
||||
|
||||
DROP TABLE 02417_test_another SYNC;
|
@ -0,0 +1,2 @@
|
||||
Ok
|
||||
Ok
|
51
tests/queries/0_stateless/02417_load_marks_async.sh
Executable file
51
tests/queries/0_stateless/02417_load_marks_async.sh
Executable file
@ -0,0 +1,51 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS test;"
|
||||
${CLICKHOUSE_CLIENT} -n -q "
|
||||
CREATE TABLE test
|
||||
(
|
||||
n0 UInt64,
|
||||
n1 UInt64,
|
||||
n2 UInt64,
|
||||
n3 UInt64,
|
||||
n4 UInt64,
|
||||
n5 UInt64,
|
||||
n6 UInt64,
|
||||
n7 UInt64,
|
||||
n8 UInt64,
|
||||
n9 UInt64
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY n0 SETTINGS min_bytes_for_wide_part = 1;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "INSERT INTO test select number, number % 3, number % 5, number % 10, number % 13, number % 15, number % 17, number % 18, number % 22, number % 25 from numbers(1000000)"
|
||||
${CLICKHOUSE_CLIENT} -q "SYSTEM STOP MERGES test"
|
||||
|
||||
function test
|
||||
{
|
||||
QUERY_ID=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(reverse(reinterpretAsString(generateUUIDv4()))))")
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "SYSTEM DROP MARK CACHE"
|
||||
${CLICKHOUSE_CLIENT} --query_id "${QUERY_ID}" -q "SELECT * FROM test SETTINGS load_marks_asynchronously=$1 FORMAT Null"
|
||||
${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS"
|
||||
|
||||
result=$(${CLICKHOUSE_CLIENT} -q "SELECT ProfileEvents['WaitMarksLoadMicroseconds'] FROM system.query_log WHERE query_id = '${QUERY_ID}' AND type = 'QueryFinish' AND current_database = currentDatabase()")
|
||||
if [[ $result -ne 0 ]]; then
|
||||
echo 'Ok'
|
||||
else
|
||||
echo 'F'
|
||||
fi
|
||||
result=$(${CLICKHOUSE_CLIENT} -q "SELECT ProfileEvents['BackgroundLoadingMarksTasks'] FROM system.query_log WHERE query_id = '${QUERY_ID}' AND type = 'QueryFinish' AND current_database = currentDatabase()")
|
||||
if [[ $result -ne 0 ]]; then
|
||||
echo 'Ok'
|
||||
else
|
||||
echo 'F'
|
||||
fi
|
||||
}
|
||||
|
||||
test 1
|
@ -0,0 +1,4 @@
|
||||
2
|
||||
3
|
||||
4
|
||||
4
|
23
tests/queries/0_stateless/02418_keeper_map_keys_limit.sql
Normal file
23
tests/queries/0_stateless/02418_keeper_map_keys_limit.sql
Normal file
@ -0,0 +1,23 @@
|
||||
-- Tags: no-ordinary-database, no-fasttest
|
||||
|
||||
DROP TABLE IF EXISTS 02418_test SYNC;
|
||||
|
||||
CREATE TABLE 02418_test (key UInt64, value Float64) Engine=KeeperMap('/' || currentDatabase() || '/test2418', 3) PRIMARY KEY(key);
|
||||
|
||||
INSERT INTO 02418_test VALUES (1, 1.1), (2, 2.2);
|
||||
SELECT count() FROM 02418_test;
|
||||
|
||||
INSERT INTO 02418_test VALUES (3, 3.3), (4, 4.4); -- { serverError 290 }
|
||||
|
||||
INSERT INTO 02418_test VALUES (1, 2.1), (2, 3.2), (3, 3.3);
|
||||
SELECT count() FROM 02418_test;
|
||||
|
||||
CREATE TABLE 02418_test_another (key UInt64, value Float64) Engine=KeeperMap('/' || currentDatabase() || '/test2418', 4) PRIMARY KEY(key);
|
||||
INSERT INTO 02418_test VALUES (4, 4.4); -- { serverError 290 }
|
||||
INSERT INTO 02418_test_another VALUES (4, 4.4);
|
||||
|
||||
SELECT count() FROM 02418_test;
|
||||
SELECT count() FROM 02418_test_another;
|
||||
|
||||
DROP TABLE 02418_test SYNC;
|
||||
DROP TABLE 02418_test_another SYNC;
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user