Merge branch 'master' into marco-vb/setting-stop-insert-on-full-disk

This commit is contained in:
Marco Vilas Boas 2024-09-13 10:44:04 +01:00 committed by GitHub
commit 8299b31d47
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
81 changed files with 2445 additions and 520 deletions

View File

@ -6,7 +6,7 @@ import SelfManaged from '@site/docs/en/_snippets/_self_managed_only_no_roadmap.m
<SelfManaged />
[SSL 'strict' option](../server-configuration-parameters/settings.md#server_configuration_parameters-openssl) enables mandatory certificate validation for the incoming connections. In this case, only connections with trusted certificates can be established. Connections with untrusted certificates will be rejected. Thus, certificate validation allows to uniquely authenticate an incoming connection. `Common Name` or `subjectAltName extension` field of the certificate is used to identify the connected user. This allows to associate multiple certificates with the same user. Additionally, reissuing and revoking of the certificates does not affect the ClickHouse configuration.
[SSL 'strict' option](../server-configuration-parameters/settings.md#server_configuration_parameters-openssl) enables mandatory certificate validation for the incoming connections. In this case, only connections with trusted certificates can be established. Connections with untrusted certificates will be rejected. Thus, certificate validation allows to uniquely authenticate an incoming connection. `Common Name` or `subjectAltName extension` field of the certificate is used to identify the connected user. `subjectAltName extension` supports the usage of one wildcard '*' in the server configuration. This allows to associate multiple certificates with the same user. Additionally, reissuing and revoking of the certificates does not affect the ClickHouse configuration.
To enable SSL certificate authentication, a list of `Common Name`'s or `Subject Alt Name`'s for each ClickHouse user must be specified in the settings file `users.xml `:
@ -30,6 +30,12 @@ To enable SSL certificate authentication, a list of `Common Name`'s or `Subject
</ssl_certificates>
<!-- Other settings -->
</user_name_2>
<user_name_3>
<ssl_certificates>
<!-- Wildcard support -->
<subject_alt_name>URI:spiffe://foo.com/*/bar</subject_alt_name>
</ssl_certificates>
</user_name_3>
</users>
</clickhouse>
```

View File

@ -55,7 +55,7 @@ keeper foo bar
- `touch '<path>'` -- Creates new node with an empty string as value. Doesn't throw an exception if the node already exists
- `get '<path>'` -- Returns the node's value
- `rm '<path>' [version]` -- Removes the node only if version matches (default: -1)
- `rmr '<path>'` -- Recursively deletes path. Confirmation required
- `rmr '<path>' [limit]` -- Recursively deletes path if the subtree size is smaller than the limit. Confirmation required (default limit = 100)
- `flwc <command>` -- Executes four-letter-word command
- `help` -- Prints this message
- `get_direct_children_number '[path]'` -- Get numbers of direct children nodes under a specific path

View File

@ -506,14 +506,23 @@ bool RMRCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & nod
return false;
node->args.push_back(std::move(path));
ASTPtr remove_nodes_limit;
if (ParserUnsignedInteger{}.parse(pos, remove_nodes_limit, expected))
node->args.push_back(remove_nodes_limit->as<ASTLiteral &>().value);
else
node->args.push_back(UInt64(100));
return true;
}
void RMRCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const
{
String path = client->getAbsolutePath(query->args[0].safeGet<String>());
UInt64 remove_nodes_limit = query->args[1].safeGet<UInt64>();
client->askConfirmation(
"You are going to recursively delete path " + path, [client, path] { client->zookeeper->removeRecursive(path); });
"You are going to recursively delete path " + path,
[client, path, remove_nodes_limit] { client->zookeeper->removeRecursive(path, static_cast<UInt32>(remove_nodes_limit)); });
}
bool ReconfigCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, DB::Expected & expected) const

View File

@ -184,7 +184,7 @@ class RMRCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override { return "{} <path> -- Recursively deletes path. Confirmation required"; }
String getHelpMessage() const override { return "{} <path> [limit] -- Recursively deletes path if the subtree size is smaller than the limit. Confirmation required (default limit = 100)"; }
};
class ReconfigCommand : public IKeeperClientCommand

View File

@ -239,15 +239,36 @@ bool Authentication::areCredentialsValid(
throw Authentication::Require<GSSAcceptorContext>(auth_data.getKerberosRealm());
case AuthenticationType::SSL_CERTIFICATE:
{
for (SSLCertificateSubjects::Type type : {SSLCertificateSubjects::Type::CN, SSLCertificateSubjects::Type::SAN})
{
for (const auto & subject : auth_data.getSSLCertificateSubjects().at(type))
{
if (ssl_certificate_credentials->getSSLCertificateSubjects().at(type).contains(subject))
return true;
// Wildcard support (1 only)
if (subject.contains('*'))
{
auto prefix = std::string_view(subject).substr(0, subject.find('*'));
auto suffix = std::string_view(subject).substr(subject.find('*') + 1);
auto slashes = std::count(subject.begin(), subject.end(), '/');
for (const auto & certificate_subject : ssl_certificate_credentials->getSSLCertificateSubjects().at(type))
{
bool matches_wildcard = certificate_subject.starts_with(prefix) && certificate_subject.ends_with(suffix);
// '*' must not represent a '/' in URI, so check if the number of '/' are equal
bool matches_slashes = slashes == count(certificate_subject.begin(), certificate_subject.end(), '/');
if (matches_wildcard && matches_slashes)
return true;
}
}
}
}
return false;
}
case AuthenticationType::SSH_KEY:
#if USE_SSH

View File

@ -171,6 +171,7 @@ bool isUserError(Error zk_return_code)
void CreateRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
void RemoveRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
void RemoveRecursiveRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
void ExistsRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
void GetRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
void SetRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }

View File

@ -248,6 +248,23 @@ struct RemoveResponse : virtual Response
{
};
struct RemoveRecursiveRequest : virtual Request
{
String path;
/// strict limit for number of deleted nodes
uint32_t remove_nodes_limit = 1;
void addRootPath(const String & root_path) override;
String getPath() const override { return path; }
size_t bytesSize() const override { return path.size() + sizeof(remove_nodes_limit); }
};
struct RemoveRecursiveResponse : virtual Response
{
};
struct ExistsRequest : virtual Request
{
String path;
@ -430,6 +447,7 @@ struct ErrorResponse : virtual Response
using CreateCallback = std::function<void(const CreateResponse &)>;
using RemoveCallback = std::function<void(const RemoveResponse &)>;
using RemoveRecursiveCallback = std::function<void(const RemoveRecursiveResponse &)>;
using ExistsCallback = std::function<void(const ExistsResponse &)>;
using GetCallback = std::function<void(const GetResponse &)>;
using SetCallback = std::function<void(const SetResponse &)>;
@ -587,6 +605,11 @@ public:
int32_t version,
RemoveCallback callback) = 0;
virtual void removeRecursive(
const String & path,
uint32_t remove_nodes_limit,
RemoveRecursiveCallback callback) = 0;
virtual void exists(
const String & path,
ExistsCallback callback,

View File

@ -90,6 +90,36 @@ struct TestKeeperRemoveRequest final : RemoveRequest, TestKeeperRequest
}
};
struct TestKeeperRemoveRecursiveRequest final : RemoveRecursiveRequest, TestKeeperRequest
{
TestKeeperRemoveRecursiveRequest() = default;
explicit TestKeeperRemoveRecursiveRequest(const RemoveRecursiveRequest & base) : RemoveRecursiveRequest(base) {}
ResponsePtr createResponse() const override;
std::pair<ResponsePtr, Undo> process(TestKeeper::Container & container, int64_t zxid) const override;
void processWatches(TestKeeper::Watches & node_watches, TestKeeper::Watches & list_watches) const override
{
std::vector<std::pair<String, size_t>> deleted;
auto add_deleted_watches = [&](TestKeeper::Watches & w)
{
for (const auto & [watch_path, _] : w)
if (watch_path.starts_with(path))
deleted.emplace_back(watch_path, std::count(watch_path.begin(), watch_path.end(), '/'));
};
add_deleted_watches(node_watches);
add_deleted_watches(list_watches);
std::sort(deleted.begin(), deleted.end(), [](const auto & lhs, const auto & rhs)
{
return lhs.second < rhs.second;
});
for (const auto & [watch_path, _] : deleted)
processWatchesImpl(watch_path, node_watches, list_watches);
}
};
struct TestKeeperExistsRequest final : ExistsRequest, TestKeeperRequest
{
ResponsePtr createResponse() const override;
@ -175,6 +205,10 @@ struct TestKeeperMultiRequest final : MultiRequest, TestKeeperRequest
{
requests.push_back(std::make_shared<TestKeeperRemoveRequest>(*concrete_request_remove));
}
else if (const auto * concrete_request_remove_recursive = dynamic_cast<const RemoveRecursiveRequest *>(generic_request.get()))
{
requests.push_back(std::make_shared<TestKeeperRemoveRecursiveRequest>(*concrete_request_remove_recursive));
}
else if (const auto * concrete_request_set = dynamic_cast<const SetRequest *>(generic_request.get()))
{
requests.push_back(std::make_shared<TestKeeperSetRequest>(*concrete_request_set));
@ -313,6 +347,62 @@ std::pair<ResponsePtr, Undo> TestKeeperRemoveRequest::process(TestKeeper::Contai
return { std::make_shared<RemoveResponse>(response), undo };
}
std::pair<ResponsePtr, Undo> TestKeeperRemoveRecursiveRequest::process(TestKeeper::Container & container, int64_t zxid) const
{
RemoveRecursiveResponse response;
response.zxid = zxid;
Undo undo;
auto root_it = container.find(path);
if (root_it == container.end())
{
response.error = Error::ZNONODE;
return { std::make_shared<RemoveRecursiveResponse>(response), undo };
}
std::vector<std::pair<std::string, Coordination::TestKeeper::Node>> children;
for (auto it = std::next(root_it); it != container.end(); ++it)
{
const auto & [child_path, child_node] = *it;
if (child_path.starts_with(path))
children.emplace_back(child_path, child_node);
else
break;
}
if (children.size() > remove_nodes_limit)
{
response.error = Error::ZNOTEMPTY;
return { std::make_shared<RemoveRecursiveResponse>(response), undo };
}
auto & parent = container.at(parentPath(path));
--parent.stat.numChildren;
++parent.stat.cversion;
for (const auto & [child_path, child_node] : children)
{
auto child_it = container.find(child_path);
chassert(child_it != container.end());
container.erase(child_it);
}
response.error = Error::ZOK;
undo = [&container, dead = std::move(children), root_path = path]()
{
for (auto && [child_path, child_node] : dead)
container.emplace(child_path, child_node);
auto & undo_parent = container.at(parentPath(root_path));
++undo_parent.stat.numChildren;
--undo_parent.stat.cversion;
};
return { std::make_shared<RemoveRecursiveResponse>(response), undo };
}
std::pair<ResponsePtr, Undo> TestKeeperExistsRequest::process(TestKeeper::Container & container, int64_t zxid) const
{
ExistsResponse response;
@ -530,6 +620,7 @@ std::pair<ResponsePtr, Undo> TestKeeperMultiRequest::process(TestKeeper::Contain
ResponsePtr TestKeeperCreateRequest::createResponse() const { return std::make_shared<CreateResponse>(); }
ResponsePtr TestKeeperRemoveRequest::createResponse() const { return std::make_shared<RemoveResponse>(); }
ResponsePtr TestKeeperRemoveRecursiveRequest::createResponse() const { return std::make_shared<RemoveRecursiveResponse>(); }
ResponsePtr TestKeeperExistsRequest::createResponse() const { return std::make_shared<ExistsResponse>(); }
ResponsePtr TestKeeperGetRequest::createResponse() const { return std::make_shared<GetResponse>(); }
ResponsePtr TestKeeperSetRequest::createResponse() const { return std::make_shared<SetResponse>(); }
@ -771,6 +862,21 @@ void TestKeeper::remove(
pushRequest(std::move(request_info));
}
void TestKeeper::removeRecursive(
const String & path,
uint32_t remove_nodes_limit,
RemoveRecursiveCallback callback)
{
TestKeeperRemoveRecursiveRequest request;
request.path = path;
request.remove_nodes_limit = remove_nodes_limit;
RequestInfo request_info;
request_info.request = std::make_shared<TestKeeperRemoveRecursiveRequest>(std::move(request));
request_info.callback = [callback](const Response & response) { callback(dynamic_cast<const RemoveRecursiveResponse &>(response)); };
pushRequest(std::move(request_info));
}
void TestKeeper::exists(
const String & path,
ExistsCallback callback,

View File

@ -58,6 +58,11 @@ public:
int32_t version,
RemoveCallback callback) override;
void removeRecursive(
const String & path,
uint32_t remove_nodes_limit,
RemoveRecursiveCallback callback) override;
void exists(
const String & path,
ExistsCallback callback,

View File

@ -31,6 +31,7 @@ using AsyncResponses = std::vector<std::pair<std::string, std::future<R>>>;
Coordination::RequestPtr makeCreateRequest(const std::string & path, const std::string & data, int create_mode, bool ignore_if_exists = false);
Coordination::RequestPtr makeRemoveRequest(const std::string & path, int version);
Coordination::RequestPtr makeRemoveRecursiveRequest(const std::string & path, uint32_t remove_nodes_limit);
Coordination::RequestPtr makeSetRequest(const std::string & path, const std::string & data, int version);
Coordination::RequestPtr makeCheckRequest(const std::string & path, int version);

View File

@ -979,18 +979,47 @@ bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probab
return removed_as_expected;
}
void ZooKeeper::removeRecursive(const std::string & path)
void ZooKeeper::removeRecursive(const std::string & path, uint32_t remove_nodes_limit)
{
removeChildrenRecursive(path);
remove(path);
if (!isFeatureEnabled(DB::KeeperFeatureFlag::REMOVE_RECURSIVE))
{
removeChildrenRecursive(path);
remove(path);
return;
}
check(tryRemoveRecursive(path, remove_nodes_limit), path);
}
void ZooKeeper::tryRemoveRecursive(const std::string & path)
Coordination::Error ZooKeeper::tryRemoveRecursive(const std::string & path, uint32_t remove_nodes_limit)
{
tryRemoveChildrenRecursive(path);
tryRemove(path);
}
if (!isFeatureEnabled(DB::KeeperFeatureFlag::REMOVE_RECURSIVE))
{
tryRemoveChildrenRecursive(path);
return tryRemove(path);
}
auto promise = std::make_shared<std::promise<Coordination::RemoveRecursiveResponse>>();
auto future = promise->get_future();
auto callback = [promise](const Coordination::RemoveRecursiveResponse & response) mutable
{
promise->set_value(response);
};
impl->removeRecursive(path, remove_nodes_limit, std::move(callback));
if (future.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready)
{
impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::RemoveRecursive, path));
return Coordination::Error::ZOPERATIONTIMEOUT;
}
else
{
auto response = future.get();
return response.error;
}
}
namespace
{
@ -1619,6 +1648,14 @@ Coordination::RequestPtr makeRemoveRequest(const std::string & path, int version
return request;
}
Coordination::RequestPtr makeRemoveRecursiveRequest(const std::string & path, uint32_t remove_nodes_limit)
{
auto request = std::make_shared<Coordination::RemoveRecursiveRequest>();
request->path = path;
request->remove_nodes_limit = remove_nodes_limit;
return request;
}
Coordination::RequestPtr makeSetRequest(const std::string & path, const std::string & data, int version)
{
auto request = std::make_shared<Coordination::SetRequest>();

View File

@ -479,15 +479,16 @@ public:
Int64 getClientID();
/// Remove the node with the subtree. If someone concurrently adds or removes a node
/// in the subtree, the result is undefined.
void removeRecursive(const std::string & path);
/// Remove the node with the subtree.
/// If Keeper supports RemoveRecursive operation then it will be performed atomically.
/// Otherwise if someone concurrently adds or removes a node in the subtree, the result is undefined.
void removeRecursive(const std::string & path, uint32_t remove_nodes_limit = 100);
/// Remove the node with the subtree. If someone concurrently removes a node in the subtree,
/// this will not cause errors.
/// Same as removeRecursive but in case if Keeper does not supports RemoveRecursive and
/// if someone concurrently removes a node in the subtree, this will not cause errors.
/// For instance, you can call this method twice concurrently for the same node and the end
/// result would be the same as for the single call.
void tryRemoveRecursive(const std::string & path);
Coordination::Error tryRemoveRecursive(const std::string & path, uint32_t remove_nodes_limit = 100);
/// Similar to removeRecursive(...) and tryRemoveRecursive(...), but does not remove path itself.
/// Node defined as RemoveException will not be deleted.

View File

@ -1,5 +1,5 @@
#include "Common/ZooKeeper/IKeeper.h"
#include "Common/ZooKeeper/ZooKeeperConstants.h"
#include <Common/ZooKeeper/IKeeper.h>
#include <Common/ZooKeeper/ZooKeeperConstants.h>
#include <Common/ZooKeeper/ZooKeeperCommon.h>
#include <Common/ZooKeeper/ZooKeeperIO.h>
#include <Common/Stopwatch.h>
@ -232,6 +232,27 @@ void ZooKeeperRemoveRequest::readImpl(ReadBuffer & in)
Coordination::read(version, in);
}
void ZooKeeperRemoveRecursiveRequest::writeImpl(WriteBuffer & out) const
{
Coordination::write(path, out);
Coordination::write(remove_nodes_limit, out);
}
void ZooKeeperRemoveRecursiveRequest::readImpl(ReadBuffer & in)
{
Coordination::read(path, in);
Coordination::read(remove_nodes_limit, in);
}
std::string ZooKeeperRemoveRecursiveRequest::toStringImpl(bool /*short_format*/) const
{
return fmt::format(
"path = {}\n"
"remove_nodes_limit = {}",
path,
remove_nodes_limit);
}
void ZooKeeperExistsRequest::writeImpl(WriteBuffer & out) const
{
Coordination::write(path, out);
@ -510,6 +531,11 @@ ZooKeeperMultiRequest::ZooKeeperMultiRequest(std::span<const Coordination::Reque
checkOperationType(Write);
requests.push_back(std::make_shared<ZooKeeperRemoveRequest>(*concrete_request_remove));
}
else if (const auto * concrete_request_remove_recursive = dynamic_cast<const RemoveRecursiveRequest *>(generic_request.get()))
{
checkOperationType(Write);
requests.push_back(std::make_shared<ZooKeeperRemoveRecursiveRequest>(*concrete_request_remove_recursive));
}
else if (const auto * concrete_request_set = dynamic_cast<const SetRequest *>(generic_request.get()))
{
checkOperationType(Write);
@ -707,6 +733,7 @@ ZooKeeperResponsePtr ZooKeeperHeartbeatRequest::makeResponse() const { return se
ZooKeeperResponsePtr ZooKeeperSyncRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperSyncResponse>()); }
ZooKeeperResponsePtr ZooKeeperAuthRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperAuthResponse>()); }
ZooKeeperResponsePtr ZooKeeperRemoveRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperRemoveResponse>()); }
ZooKeeperResponsePtr ZooKeeperRemoveRecursiveRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperRemoveRecursiveResponse>()); }
ZooKeeperResponsePtr ZooKeeperExistsRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperExistsResponse>()); }
ZooKeeperResponsePtr ZooKeeperGetRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperGetResponse>()); }
ZooKeeperResponsePtr ZooKeeperSetRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperSetResponse>()); }
@ -1024,6 +1051,7 @@ ZooKeeperRequestFactory::ZooKeeperRequestFactory()
registerZooKeeperRequest<OpNum::SetACL, ZooKeeperSetACLRequest>(*this);
registerZooKeeperRequest<OpNum::FilteredList, ZooKeeperFilteredListRequest>(*this);
registerZooKeeperRequest<OpNum::CheckNotExists, ZooKeeperCheckRequest>(*this);
registerZooKeeperRequest<OpNum::RemoveRecursive, ZooKeeperRemoveRecursiveRequest>(*this);
}
PathMatchResult matchPath(std::string_view path, std::string_view match_to)

View File

@ -285,6 +285,31 @@ struct ZooKeeperRemoveResponse final : RemoveResponse, ZooKeeperResponse
size_t bytesSize() const override { return RemoveResponse::bytesSize() + sizeof(xid) + sizeof(zxid); }
};
struct ZooKeeperRemoveRecursiveRequest final : RemoveRecursiveRequest, ZooKeeperRequest
{
ZooKeeperRemoveRecursiveRequest() = default;
explicit ZooKeeperRemoveRecursiveRequest(const RemoveRecursiveRequest & base) : RemoveRecursiveRequest(base) {}
OpNum getOpNum() const override { return OpNum::RemoveRecursive; }
void writeImpl(WriteBuffer & out) const override;
void readImpl(ReadBuffer & in) override;
std::string toStringImpl(bool short_format) const override;
ZooKeeperResponsePtr makeResponse() const override;
bool isReadRequest() const override { return false; }
size_t bytesSize() const override { return RemoveRecursiveRequest::bytesSize() + sizeof(xid); }
};
struct ZooKeeperRemoveRecursiveResponse : RemoveRecursiveResponse, ZooKeeperResponse
{
void readImpl(ReadBuffer &) override {}
void writeImpl(WriteBuffer &) const override {}
OpNum getOpNum() const override { return OpNum::RemoveRecursive; }
size_t bytesSize() const override { return RemoveRecursiveResponse::bytesSize() + sizeof(xid) + sizeof(zxid); }
};
struct ZooKeeperExistsRequest final : ExistsRequest, ZooKeeperRequest
{
ZooKeeperExistsRequest() = default;

View File

@ -29,6 +29,7 @@ static const std::unordered_set<int32_t> VALID_OPERATIONS =
static_cast<int32_t>(OpNum::GetACL),
static_cast<int32_t>(OpNum::FilteredList),
static_cast<int32_t>(OpNum::CheckNotExists),
static_cast<int32_t>(OpNum::RemoveRecursive),
};
OpNum getOpNum(int32_t raw_op_num)

View File

@ -40,6 +40,7 @@ enum class OpNum : int32_t
FilteredList = 500,
CheckNotExists = 501,
CreateIfNotExists = 502,
RemoveRecursive = 503,
SessionID = 997, /// Special internal request
};

View File

@ -1347,6 +1347,25 @@ void ZooKeeper::remove(
ProfileEvents::increment(ProfileEvents::ZooKeeperRemove);
}
void ZooKeeper::removeRecursive(
const String &path,
uint32_t remove_nodes_limit,
RemoveRecursiveCallback callback)
{
if (!isFeatureEnabled(KeeperFeatureFlag::REMOVE_RECURSIVE))
throw Exception::fromMessage(Error::ZBADARGUMENTS, "RemoveRecursive request type cannot be used because it's not supported by the server");
ZooKeeperRemoveRecursiveRequest request;
request.path = path;
request.remove_nodes_limit = remove_nodes_limit;
RequestInfo request_info;
request_info.request = std::make_shared<ZooKeeperRemoveRecursiveRequest>(std::move(request));
request_info.callback = [callback](const Response & response) { callback(dynamic_cast<const RemoveRecursiveResponse &>(response)); };
pushRequest(std::move(request_info));
ProfileEvents::increment(ProfileEvents::ZooKeeperRemove);
}
void ZooKeeper::exists(
const String & path,

View File

@ -146,6 +146,11 @@ public:
int32_t version,
RemoveCallback callback) override;
void removeRecursive(
const String &path,
uint32_t remove_nodes_limit,
RemoveRecursiveCallback callback) override;
void exists(
const String & path,
ExistsCallback callback,

View File

@ -57,11 +57,13 @@ namespace ErrorCodes
namespace DB
{
static bool supportsAtomicRenameImpl()
static std::optional<std::string> supportsAtomicRenameImpl()
{
VersionNumber renameat2_minimal_version(3, 15, 0);
VersionNumber linux_version(Poco::Environment::osVersion());
return linux_version >= renameat2_minimal_version;
if (linux_version >= renameat2_minimal_version)
return std::nullopt;
return fmt::format("Linux kernel 3.15+ is required, got {}", linux_version.toString());
}
static bool renameat2(const std::string & old_path, const std::string & new_path, int flags)
@ -97,10 +99,14 @@ static bool renameat2(const std::string & old_path, const std::string & new_path
ErrnoException::throwFromPath(ErrorCodes::SYSTEM_ERROR, new_path, "Cannot rename {} to {}", old_path, new_path);
}
bool supportsAtomicRename()
bool supportsAtomicRename(std::string * out_message)
{
static bool supports = supportsAtomicRenameImpl();
return supports;
static auto error = supportsAtomicRenameImpl();
if (!error.has_value())
return true;
if (out_message)
*out_message = error.value();
return false;
}
}
@ -152,16 +158,22 @@ static bool renameat2(const std::string & old_path, const std::string & new_path
}
static bool supportsAtomicRenameImpl()
static std::optional<std::string> supportsAtomicRenameImpl()
{
auto fun = dlsym(RTLD_DEFAULT, "renamex_np");
return fun != nullptr;
if (fun != nullptr)
return std::nullopt;
return "macOS 10.12 or later is required";
}
bool supportsAtomicRename()
bool supportsAtomicRename(std::string * out_message)
{
static bool supports = supportsAtomicRenameImpl();
return supports;
static auto error = supportsAtomicRenameImpl();
if (!error.has_value())
return true;
if (out_message)
*out_message = error.value();
return false;
}
}
@ -179,8 +191,10 @@ static bool renameat2(const std::string &, const std::string &, int)
return false;
}
bool supportsAtomicRename()
bool supportsAtomicRename(std::string * out_message)
{
if (out_message)
*out_message = "only Linux and macOS are supported";
return false;
}

View File

@ -6,7 +6,7 @@ namespace DB
{
/// Returns true, if the following functions supported by the system
bool supportsAtomicRename();
bool supportsAtomicRename(std::string * out_message = nullptr);
/// Atomically rename old_path to new_path. If new_path exists, do not overwrite it and throw exception
void renameNoReplace(const std::string & old_path, const std::string & new_path);

View File

@ -39,7 +39,7 @@ using Checksum = CityHash_v1_0_2::uint128;
/// Validate checksum of data, and if it mismatches, find out possible reason and throw exception.
static void validateChecksum(char * data, size_t size, const Checksum expected_checksum)
static void validateChecksum(char * data, size_t size, const Checksum expected_checksum, bool external_data)
{
auto calculated_checksum = CityHash_v1_0_2::CityHash128(data, size);
if (expected_checksum == calculated_checksum)
@ -64,6 +64,8 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c
"this can be caused by disk bit rot. This exception protects ClickHouse "
"from data corruption due to hardware failures.";
int error_code = external_data ? ErrorCodes::CANNOT_DECOMPRESS : ErrorCodes::CHECKSUM_DOESNT_MATCH;
auto flip_bit = [](char * buf, size_t pos)
{
buf[pos / 8] ^= 1 << pos % 8;
@ -87,7 +89,7 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c
{
message << ". The mismatch is caused by single bit flip in data block at byte " << (bit_pos / 8) << ", bit " << (bit_pos % 8) << ". "
<< message_hardware_failure;
throw Exception::createDeprecated(message.str(), ErrorCodes::CHECKSUM_DOESNT_MATCH);
throw Exception::createDeprecated(message.str(), error_code);
}
flip_bit(tmp_data, bit_pos); /// Restore
@ -102,10 +104,10 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c
{
message << ". The mismatch is caused by single bit flip in checksum. "
<< message_hardware_failure;
throw Exception::createDeprecated(message.str(), ErrorCodes::CHECKSUM_DOESNT_MATCH);
throw Exception::createDeprecated(message.str(), error_code);
}
throw Exception::createDeprecated(message.str(), ErrorCodes::CHECKSUM_DOESNT_MATCH);
throw Exception::createDeprecated(message.str(), error_code);
}
static void readHeaderAndGetCodecAndSize(
@ -151,7 +153,7 @@ static void readHeaderAndGetCodecAndSize(
"Most likely corrupted data.", size_compressed_without_checksum);
if (size_compressed_without_checksum < header_size)
throw Exception(ErrorCodes::CORRUPTED_DATA, "Can't decompress data: "
throw Exception(external_data ? ErrorCodes::CANNOT_DECOMPRESS : ErrorCodes::CORRUPTED_DATA, "Can't decompress data: "
"the compressed data size ({}, this should include header size) is less than the header size ({})",
size_compressed_without_checksum, static_cast<size_t>(header_size));
}
@ -202,7 +204,7 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed,
readBinaryLittleEndian(checksum.low64, checksum_in);
readBinaryLittleEndian(checksum.high64, checksum_in);
validateChecksum(compressed_buffer, size_compressed_without_checksum, checksum);
validateChecksum(compressed_buffer, size_compressed_without_checksum, checksum, external_data);
}
ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_compressed_without_checksum + sizeof(Checksum));
@ -247,7 +249,7 @@ size_t CompressedReadBufferBase::readCompressedDataBlockForAsynchronous(size_t &
readBinaryLittleEndian(checksum.low64, checksum_in);
readBinaryLittleEndian(checksum.high64, checksum_in);
validateChecksum(compressed_buffer, size_compressed_without_checksum, checksum);
validateChecksum(compressed_buffer, size_compressed_without_checksum, checksum, external_data);
}
ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_compressed_without_checksum + sizeof(Checksum));
@ -307,7 +309,7 @@ void CompressedReadBufferBase::decompress(BufferBase::Buffer & to, size_t size_d
UInt8 header_size = ICompressionCodec::getHeaderSize();
if (size_compressed_without_checksum < header_size)
throw Exception(ErrorCodes::CORRUPTED_DATA,
throw Exception(external_data ? ErrorCodes::CANNOT_DECOMPRESS : ErrorCodes::CORRUPTED_DATA,
"Can't decompress data: the compressed data size ({}, this should include header size) is less than the header size ({})",
size_compressed_without_checksum, static_cast<size_t>(header_size));

View File

@ -11,6 +11,7 @@ enum class KeeperApiVersion : uint8_t
WITH_FILTERED_LIST,
WITH_MULTI_READ,
WITH_CHECK_NOT_EXISTS,
WITH_REMOVE_RECURSIVE,
};
const String keeper_system_path = "/keeper";

View File

@ -91,6 +91,12 @@ bool checkIfRequestIncreaseMem(const Coordination::ZooKeeperRequestPtr & request
memory_delta -= remove_req.bytesSize();
break;
}
case Coordination::OpNum::RemoveRecursive:
{
Coordination::ZooKeeperRemoveRecursiveRequest & remove_req = dynamic_cast<Coordination::ZooKeeperRemoveRecursiveRequest &>(*sub_zk_request);
memory_delta -= remove_req.bytesSize();
break;
}
default:
break;
}

View File

@ -12,6 +12,7 @@ enum class KeeperFeatureFlag : size_t
MULTI_READ,
CHECK_NOT_EXISTS,
CREATE_IF_NOT_EXISTS,
REMOVE_RECURSIVE,
};
class KeeperFeatureFlags

View File

@ -832,6 +832,15 @@ std::shared_ptr<typename Container::Node> KeeperStorage<Container>::UncommittedS
return tryGetNodeFromStorage(path);
}
template<typename Container>
const typename Container::Node * KeeperStorage<Container>::UncommittedState::getActualNodeView(StringRef path, const Node & storage_node) const
{
if (auto node_it = nodes.find(path.toView()); node_it != nodes.end())
return node_it->second.node.get();
return &storage_node;
}
template<typename Container>
Coordination::ACLs KeeperStorage<Container>::UncommittedState::getACLs(StringRef path) const
{
@ -1124,7 +1133,7 @@ struct KeeperStorageRequestProcessor
}
virtual KeeperStorageBase::ResponsesForSessions
processWatches(KeeperStorageBase::Watches & /*watches*/, KeeperStorageBase::Watches & /*list_watches*/) const
processWatches(const Storage & /*storage*/, int64_t /*zxid*/, KeeperStorageBase::Watches & /*watches*/, KeeperStorageBase::Watches & /*list_watches*/) const
{
return {};
}
@ -1241,7 +1250,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr
using KeeperStorageRequestProcessor<Storage>::KeeperStorageRequestProcessor;
KeeperStorageBase::ResponsesForSessions
processWatches(KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override
processWatches(const Storage & /*storage*/, int64_t /*zxid*/, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override
{
return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::CREATED);
}
@ -1462,16 +1471,41 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce
}
};
namespace
{
template <typename Storage>
void addUpdateParentPzxidDelta(Storage & storage, std::vector<typename Storage::Delta> & deltas, int64_t zxid, StringRef path)
{
auto parent_path = parentNodePath(path);
if (!storage.uncommitted_state.getNode(parent_path))
return;
deltas.emplace_back(
std::string{parent_path},
zxid,
typename Storage::UpdateNodeDelta
{
[zxid](Storage::Node & parent)
{
parent.pzxid = std::max(parent.pzxid, zxid);
}
}
);
}
}
template<typename Storage>
struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestProcessor<Storage>
{
using KeeperStorageRequestProcessor<Storage>::KeeperStorageRequestProcessor;
bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override
{
return storage.checkACL(parentNodePath(this->zk_request->getPath()), Coordination::ACL::Delete, session_id, is_local);
}
using KeeperStorageRequestProcessor<Storage>::KeeperStorageRequestProcessor;
std::vector<typename Storage::Delta>
preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override
{
@ -1488,31 +1522,12 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr
return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}};
}
const auto update_parent_pzxid = [&]()
{
auto parent_path = parentNodePath(request.path);
if (!storage.uncommitted_state.getNode(parent_path))
return;
new_deltas.emplace_back(
std::string{parent_path},
zxid,
typename Storage::UpdateNodeDelta
{
[zxid](Storage::Node & parent)
{
parent.pzxid = std::max(parent.pzxid, zxid);
}
}
);
};
auto node = storage.uncommitted_state.getNode(request.path);
if (!node)
{
if (request.restored_from_zookeeper_log)
update_parent_pzxid();
addUpdateParentPzxidDelta(storage, new_deltas, zxid, request.path);
return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}};
}
else if (request.version != -1 && request.version != node->version)
@ -1521,7 +1536,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr
return {typename Storage::Delta{zxid, Coordination::Error::ZNOTEMPTY}};
if (request.restored_from_zookeeper_log)
update_parent_pzxid();
addUpdateParentPzxidDelta(storage, new_deltas, zxid, request.path);
new_deltas.emplace_back(
std::string{parentNodePath(request.path)},
@ -1552,12 +1567,318 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr
}
KeeperStorageBase::ResponsesForSessions
processWatches(KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override
processWatches(const Storage & /*storage*/, int64_t /*zxid*/, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override
{
return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::DELETED);
}
};
template<typename Storage>
struct KeeperStorageRemoveRecursiveRequestProcessor final : public KeeperStorageRequestProcessor<Storage>
{
using KeeperStorageRequestProcessor<Storage>::KeeperStorageRequestProcessor;
bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override
{
return storage.checkACL(parentNodePath(this->zk_request->getPath()), Coordination::ACL::Delete, session_id, is_local);
}
std::vector<typename Storage::Delta>
preprocess(Storage & storage, int64_t zxid, int64_t session_id, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override
{
ProfileEvents::increment(ProfileEvents::KeeperRemoveRequest);
Coordination::ZooKeeperRemoveRecursiveRequest & request = dynamic_cast<Coordination::ZooKeeperRemoveRecursiveRequest &>(*this->zk_request);
std::vector<typename Storage::Delta> new_deltas;
if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH)
{
auto error_msg = fmt::format("Trying to delete an internal Keeper path ({}) which is not allowed", request.path);
handleSystemNodeModification(keeper_context, error_msg);
return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}};
}
auto node = storage.uncommitted_state.getNode(request.path);
if (!node)
{
if (request.restored_from_zookeeper_log)
addUpdateParentPzxidDelta(storage, new_deltas, zxid, request.path);
return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}};
}
ToDeleteTreeCollector collector(storage, zxid, session_id, request.remove_nodes_limit);
auto collect_status = collector.collect(request.path, *node);
if (collect_status == ToDeleteTreeCollector::CollectStatus::NoAuth)
return {typename Storage::Delta{zxid, Coordination::Error::ZNOAUTH}};
if (collect_status == ToDeleteTreeCollector::CollectStatus::LimitExceeded)
return {typename Storage::Delta{zxid, Coordination::Error::ZNOTEMPTY}};
if (request.restored_from_zookeeper_log)
addUpdateParentPzxidDelta(storage, new_deltas, zxid, request.path);
auto delete_deltas = collector.extractDeltas();
for (const auto & delta : delete_deltas)
{
const auto * remove_delta = std::get_if<typename Storage::RemoveNodeDelta>(&delta.operation);
if (remove_delta && remove_delta->ephemeral_owner)
storage.unregisterEphemeralPath(remove_delta->ephemeral_owner, delta.path);
}
new_deltas.insert(new_deltas.end(), std::make_move_iterator(delete_deltas.begin()), std::make_move_iterator(delete_deltas.end()));
digest = storage.calculateNodesDigest(digest, new_deltas);
return new_deltas;
}
Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override
{
Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse();
Coordination::ZooKeeperRemoveRecursiveResponse & response = dynamic_cast<Coordination::ZooKeeperRemoveRecursiveResponse &>(*response_ptr);
response.error = storage.commit(zxid);
return response_ptr;
}
KeeperStorageBase::ResponsesForSessions
processWatches(const Storage & storage, int64_t zxid, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override
{
/// need to iterate over zxid deltas and update watches for deleted tree.
const auto & deltas = storage.uncommitted_state.deltas;
KeeperStorageBase::ResponsesForSessions responses;
for (auto it = deltas.rbegin(); it != deltas.rend() && it->zxid == zxid; ++it)
{
const auto * remove_delta = std::get_if<typename Storage::RemoveNodeDelta>(&it->operation);
if (remove_delta)
{
auto new_responses = processWatchesImpl(it->path, watches, list_watches, Coordination::Event::DELETED);
responses.insert(responses.end(), std::make_move_iterator(new_responses.begin()), std::make_move_iterator(new_responses.end()));
}
}
return responses;
}
private:
using SNode = typename Storage::Node;
class ToDeleteTreeCollector
{
Storage & storage;
int64_t zxid;
int64_t session_id;
uint32_t limit;
uint32_t max_level = 0;
uint32_t nodes_observed = 1; /// root node
std::unordered_map<uint32_t, std::vector<typename Storage::Delta>> by_level_deltas;
struct Step
{
String path;
std::variant<SNode, const SNode *> node;
uint32_t level;
};
enum class CollectStatus
{
Ok,
NoAuth,
LimitExceeded,
};
friend struct KeeperStorageRemoveRecursiveRequestProcessor;
public:
ToDeleteTreeCollector(Storage & storage_, int64_t zxid_, int64_t session_id_, uint32_t limit_)
: storage(storage_)
, zxid(zxid_)
, session_id(session_id_)
, limit(limit_)
{
}
CollectStatus collect(StringRef root_path, const SNode & root_node)
{
std::deque<Step> steps;
if (checkLimits(&root_node))
return CollectStatus::LimitExceeded;
steps.push_back(Step{root_path.toString(), &root_node, 0});
while (!steps.empty())
{
Step step = std::move(steps.front());
steps.pop_front();
StringRef path = step.path;
uint32_t level = step.level;
const SNode * node_ptr = nullptr;
if (auto * rdb = std::get_if<SNode>(&step.node))
node_ptr = rdb;
else
node_ptr = std::get<const SNode *>(step.node);
chassert(!path.empty());
chassert(node_ptr != nullptr);
const auto & node = *node_ptr;
auto actual_node_ptr = storage.uncommitted_state.getActualNodeView(path, node);
chassert(actual_node_ptr != nullptr); /// explicitly check that node is not deleted
if (actual_node_ptr->numChildren() > 0 && !storage.checkACL(path, Coordination::ACL::Delete, session_id, /*is_local=*/false))
return CollectStatus::NoAuth;
if (auto status = visitRocksDBNode(steps, path, level); status != CollectStatus::Ok)
return status;
if (auto status = visitMemNode(steps, path, level); status != CollectStatus::Ok)
return status;
if (auto status = visitRootAndUncommitted(steps, path, node, level); status != CollectStatus::Ok)
return status;
}
return CollectStatus::Ok;
}
std::vector<typename Storage::Delta> extractDeltas()
{
std::vector<typename Storage::Delta> deltas;
for (ssize_t level = max_level; level >= 0; --level)
{
auto & level_deltas = by_level_deltas[static_cast<uint32_t>(level)];
deltas.insert(deltas.end(), std::make_move_iterator(level_deltas.begin()), std::make_move_iterator(level_deltas.end()));
}
return std::move(deltas);
}
private:
CollectStatus visitRocksDBNode(std::deque<Step> & steps, StringRef root_path, uint32_t level)
{
if constexpr (Storage::use_rocksdb)
{
std::filesystem::path root_fs_path(root_path.toString());
auto children = storage.container.getChildren(root_path.toString());
for (auto && [child_name, child_node] : children)
{
auto child_path = (root_fs_path / child_name).generic_string();
const auto actual_child_node_ptr = storage.uncommitted_state.getActualNodeView(child_path, child_node);
if (actual_child_node_ptr == nullptr) /// node was deleted in previous step of multi transaction
continue;
if (checkLimits(actual_child_node_ptr))
return CollectStatus::LimitExceeded;
steps.push_back(Step{std::move(child_path), std::move(child_node), level + 1});
}
}
return CollectStatus::Ok;
}
CollectStatus visitMemNode(std::deque<Step> & steps, StringRef root_path, uint32_t level)
{
if constexpr (!Storage::use_rocksdb)
{
auto node_it = storage.container.find(root_path);
if (node_it == storage.container.end())
return CollectStatus::Ok;
std::filesystem::path root_fs_path(root_path.toString());
const auto & children = node_it->value.getChildren();
for (const auto & child_name : children)
{
auto child_path = (root_fs_path / child_name.toView()).generic_string();
auto child_it = storage.container.find(child_path);
chassert(child_it != storage.container.end());
const auto & child_node = child_it->value;
const auto actual_child_node_ptr = storage.uncommitted_state.getActualNodeView(child_path, child_node);
if (actual_child_node_ptr == nullptr) /// node was deleted in previous step of multi transaction
continue;
if (checkLimits(actual_child_node_ptr))
return CollectStatus::LimitExceeded;
steps.push_back(Step{std::move(child_path), &child_node, level + 1});
}
}
return CollectStatus::Ok;
}
CollectStatus visitRootAndUncommitted(std::deque<Step> & steps, StringRef root_path, const SNode & root_node, uint32_t level)
{
const auto & nodes = storage.uncommitted_state.nodes;
/// nodes are sorted by paths with level locality
auto it = nodes.upper_bound(root_path.toString() + "/");
for (; it != nodes.end() && parentNodePath(it->first) == root_path; ++it)
{
const auto actual_child_node_ptr = it->second.node.get();
if (actual_child_node_ptr == nullptr) /// node was deleted in previous step of multi transaction
continue;
if (checkLimits(actual_child_node_ptr))
return CollectStatus::LimitExceeded;
const String & child_path = it->first;
const SNode & child_node = *it->second.node;
steps.push_back(Step{child_path, &child_node, level + 1});
}
addDelta(root_path, root_node, level);
return CollectStatus::Ok;
}
void addDelta(StringRef root_path, const SNode & root_node, uint32_t level)
{
max_level = std::max(max_level, level);
by_level_deltas[level].emplace_back(
parentNodePath(root_path).toString(),
zxid,
typename Storage::UpdateNodeDelta{
[](SNode & parent)
{
++parent.cversion;
parent.decreaseNumChildren();
}
});
by_level_deltas[level].emplace_back(root_path.toString(), zxid, typename Storage::RemoveNodeDelta{root_node.version, root_node.ephemeralOwner()});
}
bool checkLimits(const SNode * node)
{
chassert(node != nullptr);
nodes_observed += node->numChildren();
return nodes_observed > limit;
}
};
};
template<typename Storage>
struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestProcessor<Storage>
{
@ -1709,7 +2030,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce
}
KeeperStorageBase::ResponsesForSessions
processWatches(typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override
processWatches(const Storage & /*storage*/, int64_t /*zxid*/, typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override
{
return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::CHANGED);
}
@ -2131,6 +2452,10 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro
check_operation_type(OperationType::Write);
concrete_requests.push_back(std::make_shared<KeeperStorageRemoveRequestProcessor<Storage>>(sub_zk_request));
break;
case Coordination::OpNum::RemoveRecursive:
check_operation_type(OperationType::Write);
concrete_requests.push_back(std::make_shared<KeeperStorageRemoveRecursiveRequestProcessor<Storage>>(sub_zk_request));
break;
case Coordination::OpNum::Set:
check_operation_type(OperationType::Write);
concrete_requests.push_back(std::make_shared<KeeperStorageSetRequestProcessor<Storage>>(sub_zk_request));
@ -2250,12 +2575,12 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro
}
KeeperStorageBase::ResponsesForSessions
processWatches(typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override
processWatches(const Storage & storage, int64_t zxid, typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override
{
typename Storage::ResponsesForSessions result;
for (const auto & generic_request : concrete_requests)
{
auto responses = generic_request->processWatches(watches, list_watches);
auto responses = generic_request->processWatches(storage, zxid, watches, list_watches);
result.insert(result.end(), responses.begin(), responses.end());
}
return result;
@ -2400,6 +2725,7 @@ KeeperStorageRequestProcessorsFactory<Storage>::KeeperStorageRequestProcessorsFa
registerKeeperRequestProcessor<Coordination::OpNum::SetACL, KeeperStorageSetACLRequestProcessor<Storage>>(*this);
registerKeeperRequestProcessor<Coordination::OpNum::GetACL, KeeperStorageGetACLRequestProcessor<Storage>>(*this);
registerKeeperRequestProcessor<Coordination::OpNum::CheckNotExists, KeeperStorageCheckRequestProcessor<Storage>>(*this);
registerKeeperRequestProcessor<Coordination::OpNum::RemoveRecursive, KeeperStorageRemoveRecursiveRequestProcessor<Storage>>(*this);
}
@ -2718,7 +3044,7 @@ KeeperStorage<Container>::ResponsesForSessions KeeperStorage<Container>::process
/// If this requests processed successfully we need to check watches
if (response->error == Coordination::Error::ZOK)
{
auto watch_responses = request_processor->processWatches(watches, list_watches);
auto watch_responses = request_processor->processWatches(*this, zxid, watches, list_watches);
results.insert(results.end(), watch_responses.begin(), watch_responses.end());
}

View File

@ -566,6 +566,7 @@ public:
void rollback(int64_t rollback_zxid);
std::shared_ptr<Node> getNode(StringRef path) const;
const Node * getActualNodeView(StringRef path, const Node & storage_node) const;
Coordination::ACLs getACLs(StringRef path) const;
void applyDelta(const Delta & delta);
@ -609,7 +610,18 @@ public:
using is_transparent = void; // required to make find() work with different type than key_type
};
mutable std::unordered_map<std::string, UncommittedNode, Hash, Equal> nodes;
struct PathCmp
{
using is_transparent = std::true_type;
auto operator()(const std::string_view a,
const std::string_view b) const
{
return a.size() < b.size() || (a.size() == b.size() && a < b);
}
};
mutable std::map<std::string, UncommittedNode, PathCmp> nodes;
std::unordered_map<std::string, std::list<const Delta *>, Hash, Equal> deltas_for_path;
std::list<Delta> deltas;

View File

@ -3113,6 +3113,8 @@ TYPED_TEST(CoordinationTest, TestFeatureFlags)
ASSERT_TRUE(feature_flags.isEnabled(KeeperFeatureFlag::FILTERED_LIST));
ASSERT_TRUE(feature_flags.isEnabled(KeeperFeatureFlag::MULTI_READ));
ASSERT_FALSE(feature_flags.isEnabled(KeeperFeatureFlag::CHECK_NOT_EXISTS));
ASSERT_FALSE(feature_flags.isEnabled(KeeperFeatureFlag::CREATE_IF_NOT_EXISTS));
ASSERT_FALSE(feature_flags.isEnabled(KeeperFeatureFlag::REMOVE_RECURSIVE));
}
TYPED_TEST(CoordinationTest, TestSystemNodeModify)
@ -3374,6 +3376,474 @@ TYPED_TEST(CoordinationTest, TestReapplyingDeltas)
ASSERT_TRUE(children1_set == children2_set);
}
TYPED_TEST(CoordinationTest, TestRemoveRecursiveRequest)
{
using namespace DB;
using namespace Coordination;
using Storage = typename TestFixture::Storage;
ChangelogDirTest rocks("./rocksdb");
this->setRocksDBDirectory("./rocksdb");
Storage storage{500, "", this->keeper_context};
int32_t zxid = 0;
const auto create = [&](const String & path, int create_mode)
{
int new_zxid = ++zxid;
const auto create_request = std::make_shared<ZooKeeperCreateRequest>();
create_request->path = path;
create_request->is_ephemeral = create_mode == zkutil::CreateMode::Ephemeral || create_mode == zkutil::CreateMode::EphemeralSequential;
create_request->is_sequential = create_mode == zkutil::CreateMode::PersistentSequential || create_mode == zkutil::CreateMode::EphemeralSequential;
storage.preprocessRequest(create_request, 1, 0, new_zxid);
auto responses = storage.processRequest(create_request, 1, new_zxid);
EXPECT_EQ(responses.size(), 1);
EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK) << "Failed to create " << path;
};
const auto remove = [&](const String & path, int32_t version = -1)
{
int new_zxid = ++zxid;
auto remove_request = std::make_shared<ZooKeeperRemoveRequest>();
remove_request->path = path;
remove_request->version = version;
storage.preprocessRequest(remove_request, 1, 0, new_zxid);
return storage.processRequest(remove_request, 1, new_zxid);
};
const auto remove_recursive = [&](const String & path, uint32_t remove_nodes_limit = 1)
{
int new_zxid = ++zxid;
auto remove_request = std::make_shared<ZooKeeperRemoveRecursiveRequest>();
remove_request->path = path;
remove_request->remove_nodes_limit = remove_nodes_limit;
storage.preprocessRequest(remove_request, 1, 0, new_zxid);
return storage.processRequest(remove_request, 1, new_zxid);
};
const auto exists = [&](const String & path)
{
int new_zxid = ++zxid;
const auto exists_request = std::make_shared<ZooKeeperExistsRequest>();
exists_request->path = path;
storage.preprocessRequest(exists_request, 1, 0, new_zxid);
auto responses = storage.processRequest(exists_request, 1, new_zxid);
EXPECT_EQ(responses.size(), 1);
return responses[0].response->error == Coordination::Error::ZOK;
};
{
SCOPED_TRACE("Single Remove Single Node");
create("/T1", zkutil::CreateMode::Persistent);
auto responses = remove("/T1");
ASSERT_EQ(responses.size(), 1);
ASSERT_EQ(responses[0].response->error, Coordination::Error::ZOK);
ASSERT_FALSE(exists("/T1"));
}
{
SCOPED_TRACE("Single Remove Tree");
create("/T2", zkutil::CreateMode::Persistent);
create("/T2/A", zkutil::CreateMode::Persistent);
auto responses = remove("/T2");
ASSERT_EQ(responses.size(), 1);
ASSERT_EQ(responses[0].response->error, Coordination::Error::ZNOTEMPTY);
ASSERT_TRUE(exists("/T2"));
}
{
SCOPED_TRACE("Recursive Remove Single Node");
create("/T3", zkutil::CreateMode::Persistent);
auto responses = remove_recursive("/T3", 100);
ASSERT_EQ(responses.size(), 1);
ASSERT_EQ(responses[0].response->error, Coordination::Error::ZOK);
ASSERT_FALSE(exists("/T3"));
}
{
SCOPED_TRACE("Recursive Remove Tree Small Limit");
create("/T5", zkutil::CreateMode::Persistent);
create("/T5/A", zkutil::CreateMode::Persistent);
create("/T5/B", zkutil::CreateMode::Persistent);
create("/T5/A/C", zkutil::CreateMode::Persistent);
auto responses = remove_recursive("/T5", 2);
ASSERT_EQ(responses.size(), 1);
ASSERT_EQ(responses[0].response->error, Coordination::Error::ZNOTEMPTY);
ASSERT_TRUE(exists("/T5"));
ASSERT_TRUE(exists("/T5/A"));
ASSERT_TRUE(exists("/T5/B"));
ASSERT_TRUE(exists("/T5/A/C"));
}
{
SCOPED_TRACE("Recursive Remove Tree Big Limit");
create("/T6", zkutil::CreateMode::Persistent);
create("/T6/A", zkutil::CreateMode::Persistent);
create("/T6/B", zkutil::CreateMode::Persistent);
create("/T6/A/C", zkutil::CreateMode::Persistent);
auto responses = remove_recursive("/T6", 4);
ASSERT_EQ(responses.size(), 1);
ASSERT_EQ(responses[0].response->error, Coordination::Error::ZOK);
ASSERT_FALSE(exists("/T6"));
ASSERT_FALSE(exists("/T6/A"));
ASSERT_FALSE(exists("/T6/B"));
ASSERT_FALSE(exists("/T6/A/C"));
}
{
SCOPED_TRACE("Recursive Remove Ephemeral");
create("/T7", zkutil::CreateMode::Ephemeral);
ASSERT_EQ(storage.ephemerals.size(), 1);
auto responses = remove_recursive("/T7", 100);
ASSERT_EQ(responses.size(), 1);
ASSERT_EQ(responses[0].response->error, Coordination::Error::ZOK);
ASSERT_EQ(storage.ephemerals.size(), 0);
ASSERT_FALSE(exists("/T7"));
}
{
SCOPED_TRACE("Recursive Remove Tree With Ephemeral");
create("/T8", zkutil::CreateMode::Persistent);
create("/T8/A", zkutil::CreateMode::Persistent);
create("/T8/B", zkutil::CreateMode::Ephemeral);
create("/T8/A/C", zkutil::CreateMode::Ephemeral);
ASSERT_EQ(storage.ephemerals.size(), 1);
auto responses = remove_recursive("/T8", 4);
ASSERT_EQ(responses.size(), 1);
ASSERT_EQ(responses[0].response->error, Coordination::Error::ZOK);
ASSERT_EQ(storage.ephemerals.size(), 0);
ASSERT_FALSE(exists("/T8"));
ASSERT_FALSE(exists("/T8/A"));
ASSERT_FALSE(exists("/T8/B"));
ASSERT_FALSE(exists("/T8/A/C"));
}
}
TYPED_TEST(CoordinationTest, TestRemoveRecursiveInMultiRequest)
{
using namespace DB;
using namespace Coordination;
using Storage = typename TestFixture::Storage;
ChangelogDirTest rocks("./rocksdb");
this->setRocksDBDirectory("./rocksdb");
Storage storage{500, "", this->keeper_context};
int zxid = 0;
auto prepare_create_tree = []()
{
return Coordination::Requests{
zkutil::makeCreateRequest("/A", "A", zkutil::CreateMode::Persistent),
zkutil::makeCreateRequest("/A/B", "B", zkutil::CreateMode::Persistent),
zkutil::makeCreateRequest("/A/C", "C", zkutil::CreateMode::Ephemeral),
zkutil::makeCreateRequest("/A/B/D", "D", zkutil::CreateMode::Ephemeral),
};
};
const auto exists = [&](const String & path)
{
int new_zxid = ++zxid;
const auto exists_request = std::make_shared<ZooKeeperExistsRequest>();
exists_request->path = path;
storage.preprocessRequest(exists_request, 1, 0, new_zxid);
auto responses = storage.processRequest(exists_request, 1, new_zxid);
EXPECT_EQ(responses.size(), 1);
return responses[0].response->error == Coordination::Error::ZOK;
};
const auto is_multi_ok = [&](Coordination::ZooKeeperResponsePtr response)
{
const auto & multi_response = dynamic_cast<Coordination::ZooKeeperMultiResponse &>(*response);
for (const auto & op_response : multi_response.responses)
if (op_response->error != Coordination::Error::ZOK)
return false;
return true;
};
{
SCOPED_TRACE("Remove In Multi Tx");
int new_zxid = ++zxid;
auto ops = prepare_create_tree();
ops.push_back(zkutil::makeRemoveRequest("/A", -1));
const auto request = std::make_shared<ZooKeeperMultiRequest>(ops, ACLs{});
storage.preprocessRequest(request, 1, 0, new_zxid);
auto responses = storage.processRequest(request, 1, new_zxid);
ops.pop_back();
ASSERT_EQ(responses.size(), 1);
ASSERT_FALSE(is_multi_ok(responses[0].response));
}
{
SCOPED_TRACE("Recursive Remove In Multi Tx");
int new_zxid = ++zxid;
auto ops = prepare_create_tree();
ops.push_back(zkutil::makeRemoveRecursiveRequest("/A", 4));
const auto request = std::make_shared<ZooKeeperMultiRequest>(ops, ACLs{});
storage.preprocessRequest(request, 1, 0, new_zxid);
auto responses = storage.processRequest(request, 1, new_zxid);
ops.pop_back();
ASSERT_EQ(responses.size(), 1);
ASSERT_TRUE(is_multi_ok(responses[0].response));
ASSERT_FALSE(exists("/A"));
ASSERT_FALSE(exists("/A/C"));
ASSERT_FALSE(exists("/A/B"));
ASSERT_FALSE(exists("/A/B/D"));
}
{
SCOPED_TRACE("Recursive Remove With Regular In Multi Tx");
int new_zxid = ++zxid;
auto ops = prepare_create_tree();
ops.push_back(zkutil::makeRemoveRequest("/A/C", -1));
ops.push_back(zkutil::makeRemoveRecursiveRequest("/A", 3));
const auto request = std::make_shared<ZooKeeperMultiRequest>(ops, ACLs{});
storage.preprocessRequest(request, 1, 0, new_zxid);
auto responses = storage.processRequest(request, 1, new_zxid);
ops.pop_back();
ops.pop_back();
ASSERT_EQ(responses.size(), 1);
ASSERT_TRUE(is_multi_ok(responses[0].response));
ASSERT_FALSE(exists("/A"));
ASSERT_FALSE(exists("/A/C"));
ASSERT_FALSE(exists("/A/B"));
ASSERT_FALSE(exists("/A/B/D"));
}
{
SCOPED_TRACE("Recursive Remove From Committed and Uncommitted states");
int create_zxid = ++zxid;
auto ops = prepare_create_tree();
/// First create nodes
const auto create_request = std::make_shared<ZooKeeperMultiRequest>(ops, ACLs{});
storage.preprocessRequest(create_request, 1, 0, create_zxid);
auto create_responses = storage.processRequest(create_request, 1, create_zxid);
ASSERT_EQ(create_responses.size(), 1);
ASSERT_TRUE(is_multi_ok(create_responses[0].response));
ASSERT_TRUE(exists("/A"));
ASSERT_TRUE(exists("/A/C"));
ASSERT_TRUE(exists("/A/B"));
ASSERT_TRUE(exists("/A/B/D"));
/// Remove node A/C as a single remove request.
/// Remove all other as remove recursive request.
/// In this case we should list storage to understand the tree topology
/// but ignore already deleted nodes in uncommitted state.
int remove_zxid = ++zxid;
ops = {
zkutil::makeRemoveRequest("/A/C", -1),
zkutil::makeRemoveRecursiveRequest("/A", 3),
};
const auto remove_request = std::make_shared<ZooKeeperMultiRequest>(ops, ACLs{});
storage.preprocessRequest(remove_request, 1, 0, remove_zxid);
auto remove_responses = storage.processRequest(remove_request, 1, remove_zxid);
ASSERT_EQ(remove_responses.size(), 1);
ASSERT_TRUE(is_multi_ok(remove_responses[0].response));
ASSERT_FALSE(exists("/A"));
ASSERT_FALSE(exists("/A/C"));
ASSERT_FALSE(exists("/A/B"));
ASSERT_FALSE(exists("/A/B/D"));
}
}
TYPED_TEST(CoordinationTest, TestRemoveRecursiveWatches)
{
using namespace DB;
using namespace Coordination;
using Storage = typename TestFixture::Storage;
ChangelogDirTest rocks("./rocksdb");
this->setRocksDBDirectory("./rocksdb");
Storage storage{500, "", this->keeper_context};
int zxid = 0;
const auto create = [&](const String & path, int create_mode)
{
int new_zxid = ++zxid;
const auto create_request = std::make_shared<ZooKeeperCreateRequest>();
create_request->path = path;
create_request->is_ephemeral = create_mode == zkutil::CreateMode::Ephemeral || create_mode == zkutil::CreateMode::EphemeralSequential;
create_request->is_sequential = create_mode == zkutil::CreateMode::PersistentSequential || create_mode == zkutil::CreateMode::EphemeralSequential;
storage.preprocessRequest(create_request, 1, 0, new_zxid);
auto responses = storage.processRequest(create_request, 1, new_zxid);
EXPECT_EQ(responses.size(), 1);
EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK) << "Failed to create " << path;
};
const auto add_watch = [&](const String & path)
{
int new_zxid = ++zxid;
const auto exists_request = std::make_shared<ZooKeeperExistsRequest>();
exists_request->path = path;
exists_request->has_watch = true;
storage.preprocessRequest(exists_request, 1, 0, new_zxid);
auto responses = storage.processRequest(exists_request, 1, new_zxid);
EXPECT_EQ(responses.size(), 1);
EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK);
};
const auto add_list_watch = [&](const String & path)
{
int new_zxid = ++zxid;
const auto list_request = std::make_shared<ZooKeeperListRequest>();
list_request->path = path;
list_request->has_watch = true;
storage.preprocessRequest(list_request, 1, 0, new_zxid);
auto responses = storage.processRequest(list_request, 1, new_zxid);
EXPECT_EQ(responses.size(), 1);
EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK);
};
create("/A", zkutil::CreateMode::Persistent);
create("/A/B", zkutil::CreateMode::Persistent);
create("/A/C", zkutil::CreateMode::Ephemeral);
create("/A/B/D", zkutil::CreateMode::Ephemeral);
add_watch("/A");
add_watch("/A/B");
add_watch("/A/C");
add_watch("/A/B/D");
add_list_watch("/A");
add_list_watch("/A/B");
ASSERT_EQ(storage.watches.size(), 4);
ASSERT_EQ(storage.list_watches.size(), 2);
int new_zxid = ++zxid;
auto remove_request = std::make_shared<ZooKeeperRemoveRecursiveRequest>();
remove_request->path = "/A";
remove_request->remove_nodes_limit = 4;
storage.preprocessRequest(remove_request, 1, 0, new_zxid);
auto responses = storage.processRequest(remove_request, 1, new_zxid);
ASSERT_EQ(responses.size(), 7);
for (size_t i = 0; i < 7; ++i)
{
ASSERT_EQ(responses[i].response->error, Coordination::Error::ZOK);
if (const auto * watch_response = dynamic_cast<Coordination::ZooKeeperWatchResponse *>(responses[i].response.get()))
ASSERT_EQ(watch_response->type, Coordination::Event::DELETED);
}
ASSERT_EQ(storage.watches.size(), 0);
ASSERT_EQ(storage.list_watches.size(), 0);
}
TYPED_TEST(CoordinationTest, TestRemoveRecursiveAcls)
{
using namespace DB;
using namespace Coordination;
using Storage = typename TestFixture::Storage;
ChangelogDirTest rocks("./rocksdb");
this->setRocksDBDirectory("./rocksdb");
Storage storage{500, "", this->keeper_context};
int zxid = 0;
{
int new_zxid = ++zxid;
String user_auth_data = "test_user:test_password";
const auto auth_request = std::make_shared<ZooKeeperAuthRequest>();
auth_request->scheme = "digest";
auth_request->data = user_auth_data;
storage.preprocessRequest(auth_request, 1, 0, new_zxid);
auto responses = storage.processRequest(auth_request, 1, new_zxid);
EXPECT_EQ(responses.size(), 1);
EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK) << "Failed to add auth to session";
}
const auto create = [&](const String & path)
{
int new_zxid = ++zxid;
const auto create_request = std::make_shared<ZooKeeperCreateRequest>();
create_request->path = path;
create_request->acls = {{.permissions = ACL::Create, .scheme = "auth", .id = ""}};
storage.preprocessRequest(create_request, 1, 0, new_zxid);
auto responses = storage.processRequest(create_request, 1, new_zxid);
EXPECT_EQ(responses.size(), 1);
EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK) << "Failed to create " << path;
};
/// Add nodes with only Create ACL
create("/A");
create("/A/B");
create("/A/C");
create("/A/B/D");
{
int new_zxid = ++zxid;
auto remove_request = std::make_shared<ZooKeeperRemoveRecursiveRequest>();
remove_request->path = "/A";
remove_request->remove_nodes_limit = 4;
storage.preprocessRequest(remove_request, 1, 0, new_zxid);
auto responses = storage.processRequest(remove_request, 1, new_zxid);
EXPECT_EQ(responses.size(), 1);
EXPECT_EQ(responses[0].response->error, Coordination::Error::ZNOAUTH);
}
}
/// INSTANTIATE_TEST_SUITE_P(CoordinationTestSuite,
/// CoordinationTest,
/// ::testing::ValuesIn(std::initializer_list<CompressionParam>{CompressionParam{true, ".zstd"}, CompressionParam{false, ""}}));

View File

@ -80,8 +80,11 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"database_replicated_allow_replicated_engine_arguments", 1, 0, "Don't allow explicit arguments by default"},
{"database_replicated_allow_explicit_uuid", 0, 0, "Added a new setting to disallow explicitly specifying table UUID"},
{"parallel_replicas_local_plan", false, false, "Use local plan for local replica in a query with parallel replicas"},
{"join_to_sort_minimum_perkey_rows", 0, 40, "The lower limit of per-key average rows in the right table to determine whether to rerange the right table by key in left or inner join. This setting ensures that the optimization is not applied for sparse table keys"},
{"join_to_sort_maximum_table_rows", 0, 10000, "The maximum number of rows in the right table to determine whether to rerange the right table by key in left or inner join"},
{"allow_experimental_join_right_table_sorting", false, false, "If it is set to true, and the conditions of `join_to_sort_minimum_perkey_rows` and `join_to_sort_maximum_table_rows` are met, rerange the right table by key to improve the performance in left or inner hash join"}
{"min_free_disk_bytes_to_throw_insert", 0, 0, "Maintain some free disk space bytes from inserts while still allowing for temporary writing."},
{"min_free_disk_ratio_to_throw_insert", 0.0, 0.0, "Maintain some free disk space bytes expressed as ratio to total disk space from inserts while still allowing for temporary writing."},
{"min_free_disk_ratio_to_throw_insert", 0.0, 0.0, "Maintain some free disk space bytes expressed as ratio to total disk space from inserts while still allowing for temporary writing."},
}
},
{"24.8",
@ -103,9 +106,6 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"type_json_skip_duplicated_paths", false, false, "Allow to skip duplicated paths during JSON parsing"},
{"allow_experimental_vector_similarity_index", false, false, "Added new setting to allow experimental vector similarity indexes"},
{"input_format_try_infer_datetimes_only_datetime64", true, false, "Allow to infer DateTime instead of DateTime64 in data formats"},
{"join_to_sort_minimum_perkey_rows", 0, 40, "The lower limit of per-key average rows in the right table to determine whether to rerange the right table by key in left or inner join. This setting ensures that the optimization is not applied for sparse table keys"},
{"join_to_sort_maximum_table_rows", 0, 10000, "The maximum number of rows in the right table to determine whether to rerange the right table by key in left or inner join."},
{"allow_experimental_join_right_table_sorting", false, false, "If it is set to true, and the conditions of `join_to_sort_minimum_perkey_rows` and `join_to_sort_maximum_table_rows` are met, rerange the right table by key to improve the performance in left or inner hash join."},
}
},
{"24.7",

View File

@ -197,8 +197,9 @@ void DatabaseAtomic::renameTable(ContextPtr local_context, const String & table_
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Moving tables between databases of different engines is not supported");
}
if (exchange && !supportsAtomicRename())
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "RENAME EXCHANGE is not supported");
std::string message;
if (exchange && !supportsAtomicRename(&message))
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "RENAME EXCHANGE is not supported because exchanging files is not supported by the OS ({})", message);
waitDatabaseStarted();

View File

@ -968,6 +968,11 @@ void InterpreterCreateQuery::validateMaterializedViewColumnsAndEngine(const ASTC
if (database && database->getEngineName() != "Atomic")
throw Exception(ErrorCodes::INCORRECT_QUERY,
"Refreshable materialized views (except with APPEND) only support Atomic database engine, but database {} has engine {}", create.getDatabase(), database->getEngineName());
std::string message;
if (!supportsAtomicRename(&message))
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"Can't create refreshable materialized view because exchanging files is not supported by the OS ({})", message);
}
Block input_block;
@ -1060,6 +1065,11 @@ namespace
void setNullTableEngine(ASTStorage & storage)
{
storage.forEachPointerToChild([](void ** ptr) mutable
{
*ptr = nullptr;
});
auto engine_ast = std::make_shared<ASTFunction>();
engine_ast->name = "Null";
engine_ast->no_empty_args = true;
@ -1146,7 +1156,9 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const
else if (getContext()->getSettingsRef().restore_replace_external_engines_to_null)
{
if (StorageFactory::instance().getStorageFeatures(create.storage->engine->name).source_access_type != AccessType::NONE)
{
setNullTableEngine(*create.storage);
}
}
return;
}

View File

@ -93,6 +93,7 @@ ColumnsDescription ZooKeeperLogElement::getColumnsDescription()
{"FilteredList", static_cast<Int16>(Coordination::OpNum::FilteredList)},
{"CheckNotExists", static_cast<Int16>(Coordination::OpNum::CheckNotExists)},
{"CreateIfNotExists", static_cast<Int16>(Coordination::OpNum::CreateIfNotExists)},
{"RemoveRecursive", static_cast<Int16>(Coordination::OpNum::RemoveRecursive)},
});
auto error_enum = getCoordinationErrorCodesEnumType();

View File

@ -1118,6 +1118,19 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
&& settings.use_query_cache
&& !internal
&& client_info.query_kind == ClientInfo::QueryKind::INITIAL_QUERY
/// Bug 67476: Avoid that the query cache stores truncated results if the query ran with a non-THROW overflow mode and hit a limit.
/// This is more workaround than a fix ... unfortunately it is hard to detect from the perspective of the query cache that the
/// query result is truncated.
&& (settings.read_overflow_mode == OverflowMode::THROW
&& settings.read_overflow_mode_leaf == OverflowMode::THROW
&& settings.group_by_overflow_mode == OverflowMode::THROW
&& settings.sort_overflow_mode == OverflowMode::THROW
&& settings.result_overflow_mode == OverflowMode::THROW
&& settings.timeout_overflow_mode == OverflowMode::THROW
&& settings.set_overflow_mode == OverflowMode::THROW
&& settings.join_overflow_mode == OverflowMode::THROW
&& settings.transfer_overflow_mode == OverflowMode::THROW
&& settings.distinct_overflow_mode == OverflowMode::THROW)
&& (ast->as<ASTSelectQuery>() || ast->as<ASTSelectWithUnionQuery>());
QueryCache::Usage query_cache_usage = QueryCache::Usage::None;

View File

@ -12,12 +12,15 @@ namespace DB
struct Settings;
class QueryStatus;
using QueryStatusPtr = std::shared_ptr<QueryStatus>;
struct ITemporaryFileLookup;
using TemporaryFileLookupPtr = std::shared_ptr<ITemporaryFileLookup>;
struct BuildQueryPipelineSettings
{
ExpressionActionsSettings actions_settings;
QueryStatusPtr process_list_element;
ProgressCallback progress_callback = nullptr;
TemporaryFileLookupPtr temporary_file_lookup;
const ExpressionActionsSettings & getActionsSettings() const { return actions_settings; }
static BuildQueryPipelineSettings fromContext(ContextPtr from);

View File

@ -0,0 +1,31 @@
#pragma once
#include <base/types.h>
#include <boost/noncopyable.hpp>
#include <vector>
#include <memory>
namespace DB
{
class WriteBuffer;
class ReadBuffer;
/// Interface for accessing temporary files by some logical name (or id).
/// While building query pipeline processors can lookup temporary files by some id and use them for writing and/or reading temporary data
/// without knowing what exactly is behind the name: local file, memory buffer, object in cloud storage, etc.
struct ITemporaryFileLookup : boost::noncopyable
{
virtual ~ITemporaryFileLookup() = default;
/// Give the caller a temporary write buffer, but don't give away the ownership.
virtual WriteBuffer & getTemporaryFileForWriting(const String & file_id) = 0;
/// Give the caller a temporary read buffer, it exclusively belongs to the caller.
/// Other callers can get their own read buffer for the same temporary file.
virtual std::unique_ptr<ReadBuffer> getTemporaryFileForReading(const String & file_id) = 0;
};
using TemporaryFileLookupPtr = std::shared_ptr<ITemporaryFileLookup>;
}

View File

@ -183,13 +183,14 @@ void ColumnGathererStream::consume(Input & input, size_t source_num)
ColumnGathererTransform::ColumnGathererTransform(
const Block & header,
size_t num_inputs,
ReadBuffer & row_sources_buf_,
std::unique_ptr<ReadBuffer> row_sources_buf_,
size_t block_preferred_size_rows_,
size_t block_preferred_size_bytes_,
bool is_result_sparse_)
: IMergingTransform<ColumnGathererStream>(
num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, /*always_read_till_end_=*/ false,
num_inputs, row_sources_buf_, block_preferred_size_rows_, block_preferred_size_bytes_, is_result_sparse_)
num_inputs, *row_sources_buf_, block_preferred_size_rows_, block_preferred_size_bytes_, is_result_sparse_)
, row_sources_buf_holder(std::move(row_sources_buf_))
, log(getLogger("ColumnGathererStream"))
{
if (header.columns() != 1)

View File

@ -115,7 +115,7 @@ public:
ColumnGathererTransform(
const Block & header,
size_t num_inputs,
ReadBuffer & row_sources_buf_,
std::unique_ptr<ReadBuffer> row_sources_buf_,
size_t block_preferred_size_rows_,
size_t block_preferred_size_bytes_,
bool is_result_sparse_);
@ -124,6 +124,8 @@ public:
protected:
void onFinish() override;
std::unique_ptr<ReadBuffer> row_sources_buf_holder; /// Keep ownership of row_sources_buf while it's in use by ColumnGathererStream.
LoggerPtr log;
};

View File

@ -2133,7 +2133,7 @@ bool TCPHandler::receiveUnexpectedData(bool throw_exception)
std::shared_ptr<ReadBuffer> maybe_compressed_in;
if (last_block_in.compression == Protocol::Compression::Enable)
maybe_compressed_in = std::make_shared<CompressedReadBuffer>(*in, /* allow_different_codecs */ true);
maybe_compressed_in = std::make_shared<CompressedReadBuffer>(*in, /* allow_different_codecs */ true, /* external_data */ query_kind != ClientInfo::QueryKind::SECONDARY_QUERY);
else
maybe_compressed_in = in;
@ -2157,7 +2157,7 @@ void TCPHandler::initBlockInput()
/// with another codec that the rest of the data. Example: data sent by Distributed tables.
if (state.compression == Protocol::Compression::Enable)
state.maybe_compressed_in = std::make_shared<CompressedReadBuffer>(*in, /* allow_different_codecs */ true);
state.maybe_compressed_in = std::make_shared<CompressedReadBuffer>(*in, /* allow_different_codecs */ true, /* external_data */ query_kind != ClientInfo::QueryKind::SECONDARY_QUERY);
else
state.maybe_compressed_in = in;

View File

@ -14,7 +14,7 @@
#include <Compression/CompressedWriteBuffer.h>
#include <DataTypes/ObjectUtils.h>
#include <DataTypes/Serializations/SerializationInfo.h>
#include <IO/IReadableWriteBuffer.h>
#include <IO/ReadBufferFromEmptyFile.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/IMergeTreeDataPart.h>
#include <Storages/MergeTree/MergeTreeSequentialSource.h>
@ -43,6 +43,7 @@
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/UnionStep.h>
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/TemporaryFiles.h>
#include <Interpreters/PreparedSets.h>
#include <Interpreters/MergeTreeTransaction.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
@ -90,6 +91,68 @@ static ColumnsStatistics getStatisticsForColumns(
return all_statistics;
}
/// Manages the "rows_sources" temporary file that is used during vertical merge.
class RowsSourcesTemporaryFile : public ITemporaryFileLookup
{
public:
/// A logical name of the temporary file under which it will be known to the plan steps that use it.
static constexpr auto FILE_ID = "rows_sources";
explicit RowsSourcesTemporaryFile(TemporaryDataOnDiskScopePtr temporary_data_on_disk_)
: tmp_disk(std::make_unique<TemporaryDataOnDisk>(temporary_data_on_disk_))
, uncompressed_write_buffer(tmp_disk->createRawStream())
, tmp_file_name_on_disk(uncompressed_write_buffer->getFileName())
{
}
WriteBuffer & getTemporaryFileForWriting(const String & name) override
{
if (name != FILE_ID)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected temporary file name requested: {}", name);
if (write_buffer)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary file was already requested for writing, there musto be only one writer");
write_buffer = (std::make_unique<CompressedWriteBuffer>(*uncompressed_write_buffer));
return *write_buffer;
}
std::unique_ptr<ReadBuffer> getTemporaryFileForReading(const String & name) override
{
if (name != FILE_ID)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected temporary file name requested: {}", name);
if (!finalized)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary file is not finalized yet");
/// tmp_disk might not create real file if no data was written to it.
if (final_size == 0)
return std::make_unique<ReadBufferFromEmptyFile>();
/// Reopen the file for each read so that multiple reads can be performed in parallel and there is no need to seek to the beginning.
auto raw_file_read_buffer = std::make_unique<ReadBufferFromFile>(tmp_file_name_on_disk);
return std::make_unique<CompressedReadBufferFromFile>(std::move(raw_file_read_buffer));
}
/// Returns written data size in bytes
size_t finalizeWriting()
{
write_buffer->finalize();
uncompressed_write_buffer->finalize();
finalized = true;
final_size = write_buffer->count();
return final_size;
}
private:
std::unique_ptr<TemporaryDataOnDisk> tmp_disk;
std::unique_ptr<WriteBufferFromFileBase> uncompressed_write_buffer;
std::unique_ptr<WriteBuffer> write_buffer;
const String tmp_file_name_on_disk;
bool finalized = false;
size_t final_size = 0;
};
static void addMissedColumnsToSerializationInfos(
size_t num_rows_in_parts,
const Names & part_columns,
@ -125,19 +188,19 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColu
std::set<String> key_columns(sort_key_columns_vec.cbegin(), sort_key_columns_vec.cend());
/// Force sign column for Collapsing mode
if (ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing)
key_columns.emplace(ctx->merging_params.sign_column);
if (global_ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing)
key_columns.emplace(global_ctx->merging_params.sign_column);
/// Force version column for Replacing mode
if (ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing)
if (global_ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing)
{
key_columns.emplace(ctx->merging_params.is_deleted_column);
key_columns.emplace(ctx->merging_params.version_column);
key_columns.emplace(global_ctx->merging_params.is_deleted_column);
key_columns.emplace(global_ctx->merging_params.version_column);
}
/// Force sign column for VersionedCollapsing mode. Version is already in primary key.
if (ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing)
key_columns.emplace(ctx->merging_params.sign_column);
if (global_ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing)
key_columns.emplace(global_ctx->merging_params.sign_column);
/// Force to merge at least one column in case of empty key
if (key_columns.empty())
@ -206,7 +269,8 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const
// E.g. `proj_a.proj` for a normal projection merge and `proj_a.tmp_proj` for a projection materialization merge.
local_tmp_prefix = global_ctx->parent_part ? "" : "tmp_merge_";
}
const String local_tmp_suffix = global_ctx->parent_part ? ctx->suffix : "";
const String local_tmp_suffix = global_ctx->parent_part ? global_ctx->suffix : "";
if (global_ctx->merges_blocker->isCancelled() || global_ctx->merge_list_element_ptr->is_cancelled.load(std::memory_order_relaxed))
throw Exception(ErrorCodes::ABORTED, "Cancelled merging parts");
@ -231,7 +295,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const
LOG_DEBUG(ctx->log, "DEDUPLICATE BY ('{}')", fmt::join(global_ctx->deduplicate_by_columns, "', '"));
}
ctx->disk = global_ctx->space_reservation->getDisk();
global_ctx->disk = global_ctx->space_reservation->getDisk();
auto local_tmp_part_basename = local_tmp_prefix + global_ctx->future_part->name + local_tmp_suffix;
std::optional<MergeTreeDataPartBuilder> builder;
@ -243,7 +307,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const
}
else
{
auto local_single_disk_volume = std::make_shared<SingleDiskVolume>("volume_" + global_ctx->future_part->name, ctx->disk, 0);
auto local_single_disk_volume = std::make_shared<SingleDiskVolume>("volume_" + global_ctx->future_part->name, global_ctx->disk, 0);
builder.emplace(global_ctx->data->getDataPartBuilder(global_ctx->future_part->name, local_single_disk_volume, local_tmp_part_basename));
builder->withPartStorageType(global_ctx->future_part->part_format.storage_type);
}
@ -364,8 +428,6 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const
ctx->compression_codec = global_ctx->data->getCompressionCodecForPart(
global_ctx->merge_list_element_ptr->total_size_bytes_compressed, global_ctx->new_data_part->ttl_infos, global_ctx->time_of_merge);
ctx->tmp_disk = std::make_unique<TemporaryDataOnDisk>(global_ctx->context->getTempDataOnDisk());
switch (global_ctx->chosen_merge_algorithm)
{
case MergeAlgorithm::Horizontal:
@ -378,8 +440,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const
}
case MergeAlgorithm::Vertical:
{
ctx->rows_sources_uncompressed_write_buf = ctx->tmp_disk->createRawStream();
ctx->rows_sources_write_buf = std::make_unique<CompressedWriteBuffer>(*ctx->rows_sources_uncompressed_write_buf);
ctx->rows_sources_temporary_file = std::make_shared<RowsSourcesTemporaryFile>(global_ctx->context->getTempDataOnDisk());
std::map<String, UInt64> local_merged_column_to_size;
for (const auto & part : global_ctx->future_part->parts)
@ -499,11 +560,9 @@ MergeTask::StageRuntimeContextPtr MergeTask::ExecuteAndFinalizeHorizontalPart::g
auto new_ctx = std::make_shared<VerticalMergeRuntimeContext>();
new_ctx->rows_sources_write_buf = std::move(ctx->rows_sources_write_buf);
new_ctx->rows_sources_uncompressed_write_buf = std::move(ctx->rows_sources_uncompressed_write_buf);
new_ctx->rows_sources_temporary_file = std::move(ctx->rows_sources_temporary_file);
new_ctx->column_sizes = std::move(ctx->column_sizes);
new_ctx->compression_codec = std::move(ctx->compression_codec);
new_ctx->tmp_disk = std::move(ctx->tmp_disk);
new_ctx->it_name_and_type = std::move(ctx->it_name_and_type);
new_ctx->read_with_direct_io = std::move(ctx->read_with_direct_io);
new_ctx->need_sync = std::move(ctx->need_sync);
@ -559,9 +618,9 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::prepareProjectionsToMergeAndRe
const bool merge_may_reduce_rows =
global_ctx->cleanup ||
global_ctx->deduplicate ||
ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing ||
ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing ||
ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing;
global_ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing ||
global_ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing ||
global_ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing;
const auto & projections = global_ctx->metadata_snapshot->getProjections();
@ -760,11 +819,7 @@ bool MergeTask::VerticalMergeStage::prepareVerticalMergeForAllColumns() const
global_ctx->merge_list_element_ptr->progress.store(ctx->column_sizes->keyColumnsWeight(), std::memory_order_relaxed);
/// Ensure data has written to disk.
ctx->rows_sources_write_buf->finalize();
ctx->rows_sources_uncompressed_write_buf->finalize();
ctx->rows_sources_uncompressed_write_buf->finalize();
size_t rows_sources_count = ctx->rows_sources_write_buf->count();
size_t rows_sources_count = ctx->rows_sources_temporary_file->finalizeWriting();
/// In special case, when there is only one source part, and no rows were skipped, we may have
/// skipped writing rows_sources file. Otherwise rows_sources_count must be equal to the total
/// number of input rows.
@ -775,29 +830,6 @@ bool MergeTask::VerticalMergeStage::prepareVerticalMergeForAllColumns() const
"of bytes written to rows_sources file ({}). It is a bug.",
sum_input_rows_exact, input_rows_filtered, rows_sources_count);
/// TemporaryDataOnDisk::createRawStream returns WriteBufferFromFile implementing IReadableWriteBuffer
/// and we expect to get ReadBufferFromFile here.
/// So, it's relatively safe to use dynamic_cast here and downcast to ReadBufferFromFile.
auto * wbuf_readable = dynamic_cast<IReadableWriteBuffer *>(ctx->rows_sources_uncompressed_write_buf.get());
std::unique_ptr<ReadBuffer> reread_buf = wbuf_readable ? wbuf_readable->tryGetReadBuffer() : nullptr;
if (!reread_buf)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot read temporary file {}", ctx->rows_sources_uncompressed_write_buf->getFileName());
auto * reread_buffer_raw = dynamic_cast<ReadBufferFromFileBase *>(reread_buf.get());
if (!reread_buffer_raw)
{
const auto & reread_buf_ref = *reread_buf;
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected ReadBufferFromFileBase, but got {}", demangle(typeid(reread_buf_ref).name()));
}
/// Move ownership from std::unique_ptr<ReadBuffer> to std::unique_ptr<ReadBufferFromFile> for CompressedReadBufferFromFile.
/// First, release ownership from unique_ptr to base type.
reread_buf.release(); /// NOLINT(bugprone-unused-return-value,hicpp-ignored-remove-result): we already have the pointer value in `reread_buffer_raw`
/// Then, move ownership to unique_ptr to concrete type.
std::unique_ptr<ReadBufferFromFileBase> reread_buffer_from_file(reread_buffer_raw);
/// CompressedReadBufferFromFile expects std::unique_ptr<ReadBufferFromFile> as argument.
ctx->rows_sources_read_buf = std::make_unique<CompressedReadBufferFromFile>(std::move(reread_buffer_from_file));
ctx->it_name_and_type = global_ctx->gathering_columns.cbegin();
const auto & settings = global_ctx->context->getSettingsRef();
@ -828,12 +860,12 @@ class ColumnGathererStep : public ITransformingStep
public:
ColumnGathererStep(
const DataStream & input_stream_,
CompressedReadBufferFromFile * rows_sources_read_buf_,
const String & rows_sources_temporary_file_name_,
UInt64 merge_block_size_rows_,
UInt64 merge_block_size_bytes_,
bool is_result_sparse_)
: ITransformingStep(input_stream_, input_stream_.header, getTraits())
, rows_sources_read_buf(rows_sources_read_buf_)
, rows_sources_temporary_file_name(rows_sources_temporary_file_name_)
, merge_block_size_rows(merge_block_size_rows_)
, merge_block_size_bytes(merge_block_size_bytes_)
, is_result_sparse(is_result_sparse_)
@ -841,17 +873,20 @@ public:
String getName() const override { return "ColumnGatherer"; }
void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & /*pipelineSettings*/) override
void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & pipeline_settings) override
{
const auto &header = pipeline.getHeader();
const auto input_streams_count = pipeline.getNumStreams();
rows_sources_read_buf->seek(0, 0);
if (!pipeline_settings.temporary_file_lookup)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary file lookup is not set in pipeline settings for vertical merge");
auto rows_sources_read_buf = pipeline_settings.temporary_file_lookup->getTemporaryFileForReading(rows_sources_temporary_file_name);
auto transform = std::make_unique<ColumnGathererTransform>(
header,
input_streams_count,
*rows_sources_read_buf,
std::move(rows_sources_read_buf),
merge_block_size_rows,
merge_block_size_bytes,
is_result_sparse);
@ -881,7 +916,7 @@ private:
}
MergeTreeData::MergingParams merging_params{};
CompressedReadBufferFromFile * rows_sources_read_buf;
const String rows_sources_temporary_file_name;
const UInt64 merge_block_size_rows;
const UInt64 merge_block_size_bytes;
const bool is_result_sparse;
@ -932,7 +967,7 @@ MergeTask::VerticalMergeRuntimeContext::PreparedColumnPipeline MergeTask::Vertic
const auto data_settings = global_ctx->data->getSettings();
auto merge_step = std::make_unique<ColumnGathererStep>(
merge_column_query_plan.getCurrentDataStream(),
ctx->rows_sources_read_buf.get(), //global_ctx->rows_sources_temporary_file_name,
RowsSourcesTemporaryFile::FILE_ID,
data_settings->merge_max_block_size,
data_settings->merge_max_block_size_bytes,
is_result_sparse);
@ -961,7 +996,8 @@ MergeTask::VerticalMergeRuntimeContext::PreparedColumnPipeline MergeTask::Vertic
}
auto pipeline_settings = BuildQueryPipelineSettings::fromContext(global_ctx->context);
auto optimization_settings = QueryPlanOptimizationSettings::fromContext(global_ctx->context);
pipeline_settings.temporary_file_lookup = ctx->rows_sources_temporary_file;
auto optimization_settings = QueryPlanOptimizationSettings::fromContext(global_ctx->context);
auto builder = merge_column_query_plan.buildQueryPipeline(optimization_settings, pipeline_settings);
return {QueryPipelineBuilder::getPipeline(std::move(*builder)), std::move(indexes_to_recalc)};
@ -1013,10 +1049,6 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const
global_ctx->to->getIndexGranularity());
ctx->column_elems_written = 0;
/// rows_sources_read_buf is reused for each column so we need to rewind it explicitly each time
/// This sharing also prevents from from running multiple merge of individual columns in parallel.
ctx->rows_sources_read_buf->seek(0, 0);
}
@ -1330,7 +1362,7 @@ public:
const SortDescription & sort_description_,
const Names partition_key_columns_,
const MergeTreeData::MergingParams & merging_params_,
WriteBuffer * rows_sources_write_buf_,
const String & rows_sources_temporary_file_name_,
UInt64 merge_block_size_rows_,
UInt64 merge_block_size_bytes_,
bool blocks_are_granules_size_,
@ -1340,7 +1372,7 @@ public:
, sort_description(sort_description_)
, partition_key_columns(partition_key_columns_)
, merging_params(merging_params_)
, rows_sources_write_buf(rows_sources_write_buf_)
, rows_sources_temporary_file_name(rows_sources_temporary_file_name_)
, merge_block_size_rows(merge_block_size_rows_)
, merge_block_size_bytes(merge_block_size_bytes_)
, blocks_are_granules_size(blocks_are_granules_size_)
@ -1350,7 +1382,7 @@ public:
String getName() const override { return "MergeParts"; }
void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & /*pipelineSettings*/) override
void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & pipeline_settings) override
{
/// The order of the streams is important: when the key is matched, the elements go in the order of the source stream number.
/// In the merged part, the lines with the same key must be in the ascending order of the identifier of original part,
@ -1360,6 +1392,14 @@ public:
const auto &header = pipeline.getHeader();
const auto input_streams_count = pipeline.getNumStreams();
WriteBuffer * rows_sources_write_buf = nullptr;
if (!rows_sources_temporary_file_name.empty())
{
if (!pipeline_settings.temporary_file_lookup)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary file lookup is not set in pipeline settings for vertical merge");
rows_sources_write_buf = &pipeline_settings.temporary_file_lookup->getTemporaryFileForWriting(rows_sources_temporary_file_name);
}
switch (merging_params.mode)
{
case MergeTreeData::MergingParams::Ordinary:
@ -1449,7 +1489,7 @@ private:
const SortDescription sort_description;
const Names partition_key_columns;
const MergeTreeData::MergingParams merging_params{};
WriteBuffer * rows_sources_write_buf;
const String rows_sources_temporary_file_name;
const UInt64 merge_block_size_rows;
const UInt64 merge_block_size_bytes;
const bool blocks_are_granules_size;
@ -1606,8 +1646,9 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const
for (size_t i = 0; i < sort_columns_size; ++i)
sort_description.emplace_back(sort_columns[i], 1, 1);
const bool is_vertical_merge = (global_ctx->chosen_merge_algorithm == MergeAlgorithm::Vertical);
/// If merge is vertical we cannot calculate it
ctx->blocks_are_granules_size = (global_ctx->chosen_merge_algorithm == MergeAlgorithm::Vertical);
ctx->blocks_are_granules_size = is_vertical_merge;
if (global_ctx->cleanup && !data_settings->allow_experimental_replacing_merge_with_cleanup)
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental merges with CLEANUP are not allowed");
@ -1616,8 +1657,8 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const
merge_parts_query_plan.getCurrentDataStream(),
sort_description,
partition_key_columns,
ctx->merging_params,
ctx->rows_sources_write_buf.get(),
global_ctx->merging_params,
(is_vertical_merge ? RowsSourcesTemporaryFile::FILE_ID : ""), /// rows_sources temporaty file is used only for vertical merge
data_settings->merge_max_block_size,
data_settings->merge_max_block_size_bytes,
ctx->blocks_are_granules_size,
@ -1682,7 +1723,8 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const
{
auto pipeline_settings = BuildQueryPipelineSettings::fromContext(global_ctx->context);
auto optimization_settings = QueryPlanOptimizationSettings::fromContext(global_ctx->context);
pipeline_settings.temporary_file_lookup = ctx->rows_sources_temporary_file;
auto optimization_settings = QueryPlanOptimizationSettings::fromContext(global_ctx->context);
auto builder = merge_parts_query_plan.buildQueryPipeline(optimization_settings, pipeline_settings);
global_ctx->merged_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder));
@ -1726,10 +1768,10 @@ MergeAlgorithm MergeTask::ExecuteAndFinalizeHorizontalPart::chooseMergeAlgorithm
}
bool is_supported_storage =
ctx->merging_params.mode == MergeTreeData::MergingParams::Ordinary ||
ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing ||
ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing ||
ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing;
global_ctx->merging_params.mode == MergeTreeData::MergingParams::Ordinary ||
global_ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing ||
global_ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing ||
global_ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing;
bool enough_ordinary_cols = global_ctx->gathering_columns.size() >= data_settings->vertical_merge_algorithm_min_columns_to_activate;

View File

@ -40,6 +40,7 @@ namespace DB
class MergeTask;
using MergeTaskPtr = std::shared_ptr<MergeTask>;
class RowsSourcesTemporaryFile;
/**
* Overview of the merge algorithm
@ -100,6 +101,7 @@ public:
global_ctx->context = std::move(context_);
global_ctx->holder = &holder;
global_ctx->space_reservation = std::move(space_reservation_);
global_ctx->disk = global_ctx->space_reservation->getDisk();
global_ctx->deduplicate = std::move(deduplicate_);
global_ctx->deduplicate_by_columns = std::move(deduplicate_by_columns_);
global_ctx->cleanup = std::move(cleanup_);
@ -110,12 +112,10 @@ public:
global_ctx->ttl_merges_blocker = std::move(ttl_merges_blocker_);
global_ctx->txn = std::move(txn);
global_ctx->need_prefix = need_prefix;
global_ctx->suffix = std::move(suffix_);
global_ctx->merging_params = std::move(merging_params_);
auto prepare_stage_ctx = std::make_shared<ExecuteAndFinalizeHorizontalPartRuntimeContext>();
prepare_stage_ctx->suffix = std::move(suffix_);
prepare_stage_ctx->merging_params = std::move(merging_params_);
(*stages.begin())->setRuntimeContext(std::move(prepare_stage_ctx), global_ctx);
}
@ -172,6 +172,7 @@ private:
ContextPtr context{nullptr};
time_t time_of_merge{0};
ReservationSharedPtr space_reservation{nullptr};
DiskPtr disk{nullptr};
bool deduplicate{false};
Names deduplicate_by_columns{};
bool cleanup{false};
@ -210,6 +211,8 @@ private:
MergeTreeTransactionPtr txn;
bool need_prefix;
String suffix;
MergeTreeData::MergingParams merging_params{};
scope_guard temporary_directory_lock;
UInt64 prev_elapsed_ms{0};
@ -222,19 +225,11 @@ private:
/// Proper initialization is responsibility of the author
struct ExecuteAndFinalizeHorizontalPartRuntimeContext : public IStageRuntimeContext
{
/// Dependencies
String suffix;
bool need_prefix;
MergeTreeData::MergingParams merging_params{};
TemporaryDataOnDiskPtr tmp_disk{nullptr};
DiskPtr disk{nullptr};
bool need_remove_expired_values{false};
bool force_ttl{false};
CompressionCodecPtr compression_codec{nullptr};
size_t sum_input_rows_upper_bound{0};
std::unique_ptr<WriteBufferFromFileBase> rows_sources_uncompressed_write_buf{nullptr};
std::unique_ptr<WriteBuffer> rows_sources_write_buf{nullptr};
std::shared_ptr<RowsSourcesTemporaryFile> rows_sources_temporary_file;
std::optional<ColumnSizeEstimator> column_sizes{};
/// For projections to rebuild
@ -264,7 +259,6 @@ private:
using ExecuteAndFinalizeHorizontalPartRuntimeContextPtr = std::shared_ptr<ExecuteAndFinalizeHorizontalPartRuntimeContext>;
struct ExecuteAndFinalizeHorizontalPart : public IStage
{
bool execute() override;
@ -314,11 +308,9 @@ private:
struct VerticalMergeRuntimeContext : public IStageRuntimeContext
{
/// Begin dependencies from previous stage
std::unique_ptr<WriteBufferFromFileBase> rows_sources_uncompressed_write_buf{nullptr};
std::unique_ptr<WriteBuffer> rows_sources_write_buf{nullptr};
std::shared_ptr<RowsSourcesTemporaryFile> rows_sources_temporary_file;
std::optional<ColumnSizeEstimator> column_sizes;
CompressionCodecPtr compression_codec;
TemporaryDataOnDiskPtr tmp_disk{nullptr};
std::list<DB::NameAndTypePair>::const_iterator it_name_and_type;
bool read_with_direct_io{false};
bool need_sync{false};
@ -350,13 +342,11 @@ private:
size_t column_elems_written{0};
QueryPipeline column_parts_pipeline;
std::unique_ptr<PullingPipelineExecutor> executor;
std::unique_ptr<CompressedReadBufferFromFile> rows_sources_read_buf{nullptr};
UInt64 elapsed_execute_ns{0};
};
using VerticalMergeRuntimeContextPtr = std::shared_ptr<VerticalMergeRuntimeContext>;
struct VerticalMergeStage : public IStage
{
bool execute() override;

View File

@ -24,6 +24,7 @@ namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int LOGICAL_ERROR;
}
const std::unordered_set<std::string_view> required_configuration_keys = {
@ -146,14 +147,13 @@ void StorageAzureConfiguration::fromNamedCollection(const NamedCollection & coll
void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context, bool with_structure)
{
if (engine_args.size() < 3 || engine_args.size() > (with_structure ? 8 : 7))
if (engine_args.size() < 3 || engine_args.size() > getMaxNumberOfArguments(with_structure))
{
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Storage AzureBlobStorage requires 3 to {} arguments: "
"AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, "
"[account_name, account_key, format, compression, structure)])",
(with_structure ? 8 : 7));
"Storage AzureBlobStorage requires 1 to {} arguments. All supported signatures:\n{}",
getMaxNumberOfArguments(with_structure),
getSignatures(with_structure));
}
for (auto & engine_arg : engine_args)
@ -272,26 +272,30 @@ void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context,
connection_params = getConnectionParams(connection_url, container_name, account_name, account_key, context);
}
void StorageAzureConfiguration::addStructureAndFormatToArgs(
void StorageAzureConfiguration::addStructureAndFormatToArgsIfNeeded(
ASTs & args, const String & structure_, const String & format_, ContextPtr context)
{
if (tryGetNamedCollectionWithOverrides(args, context))
if (auto collection = tryGetNamedCollectionWithOverrides(args, context))
{
/// In case of named collection, just add key-value pair "structure='...'"
/// at the end of arguments to override existed structure.
ASTs equal_func_args = {std::make_shared<ASTIdentifier>("structure"), std::make_shared<ASTLiteral>(structure_)};
auto equal_func = makeASTFunction("equals", std::move(equal_func_args));
args.push_back(equal_func);
/// In case of named collection, just add key-value pairs "format='...', structure='...'"
/// at the end of arguments to override existed format and structure with "auto" values.
if (collection->getOrDefault<String>("format", "auto") == "auto")
{
ASTs format_equal_func_args = {std::make_shared<ASTIdentifier>("format"), std::make_shared<ASTLiteral>(format_)};
auto format_equal_func = makeASTFunction("equals", std::move(format_equal_func_args));
args.push_back(format_equal_func);
}
if (collection->getOrDefault<String>("structure", "auto") == "auto")
{
ASTs structure_equal_func_args = {std::make_shared<ASTIdentifier>("structure"), std::make_shared<ASTLiteral>(structure_)};
auto structure_equal_func = makeASTFunction("equals", std::move(structure_equal_func_args));
args.push_back(structure_equal_func);
}
}
else
{
if (args.size() < 3 || args.size() > 8)
{
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Storage Azure requires 3 to 7 arguments: "
"StorageObjectStorage(connection_string|storage_account_url, container_name, "
"blobpath, [account_name, account_key, format, compression, structure])");
}
if (args.size() < 3 || args.size() > getMaxNumberOfArguments())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 3 to {} arguments in table function azureBlobStorage, got {}", getMaxNumberOfArguments(), args.size());
for (auto & arg : args)
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);

View File

@ -22,6 +22,29 @@ public:
static constexpr auto type_name = "azure";
static constexpr auto engine_name = "Azure";
/// All possible signatures for Azure engine with structure argument (for example for azureBlobStorage table function).
static constexpr auto max_number_of_arguments_with_structure = 8;
static constexpr auto signatures_with_structure =
" - connection_string, container_name, blobpath\n"
" - connection_string, container_name, blobpath, structure \n"
" - connection_string, container_name, blobpath, format \n"
" - connection_string, container_name, blobpath, format, compression \n"
" - connection_string, container_name, blobpath, format, compression, structure \n"
" - storage_account_url, container_name, blobpath, account_name, account_key\n"
" - storage_account_url, container_name, blobpath, account_name, account_key, structure\n"
" - storage_account_url, container_name, blobpath, account_name, account_key, format\n"
" - storage_account_url, container_name, blobpath, account_name, account_key, format, compression\n"
" - storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure\n";
/// All possible signatures for Azure engine without structure argument (for example for AzureBlobStorage table engine).
static constexpr auto max_number_of_arguments_without_structure = 7;
static constexpr auto signatures_without_structure =
" - connection_string, container_name, blobpath\n"
" - connection_string, container_name, blobpath, format \n"
" - connection_string, container_name, blobpath, format, compression \n"
" - storage_account_url, container_name, blobpath, account_name, account_key\n"
" - storage_account_url, container_name, blobpath, account_name, account_key, format\n"
" - storage_account_url, container_name, blobpath, account_name, account_key, format, compression\n";
StorageAzureConfiguration() = default;
StorageAzureConfiguration(const StorageAzureConfiguration & other);
@ -29,6 +52,9 @@ public:
std::string getTypeName() const override { return type_name; }
std::string getEngineName() const override { return engine_name; }
std::string getSignatures(bool with_structure = true) const { return with_structure ? signatures_with_structure : signatures_without_structure; }
size_t getMaxNumberOfArguments(bool with_structure = true) const { return with_structure ? max_number_of_arguments_with_structure : max_number_of_arguments_without_structure; }
Path getPath() const override { return blob_path; }
void setPath(const Path & path) override { blob_path = path; }
@ -44,7 +70,7 @@ public:
ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly) override;
void addStructureAndFormatToArgs(
void addStructureAndFormatToArgsIfNeeded(
ASTs & args,
const String & structure_,
const String & format_,

View File

@ -24,6 +24,7 @@ namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int LOGICAL_ERROR;
}
StorageHDFSConfiguration::StorageHDFSConfiguration(const StorageHDFSConfiguration & other)
@ -83,12 +84,13 @@ StorageObjectStorage::QuerySettings StorageHDFSConfiguration::getQuerySettings(c
void StorageHDFSConfiguration::fromAST(ASTs & args, ContextPtr context, bool with_structure)
{
const size_t max_args_num = with_structure ? 4 : 3;
if (args.empty() || args.size() > max_args_num)
{
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Expected not more than {} arguments", max_args_num);
}
if (args.empty() || args.size() > getMaxNumberOfArguments(with_structure))
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Storage HDFS requires 1 to {} arguments. All supported signatures:\n{}",
getMaxNumberOfArguments(with_structure),
getSignatures(with_structure));
std::string url_str;
url_str = checkAndGetLiteralArgument<String>(args[0], "url");
@ -158,28 +160,34 @@ void StorageHDFSConfiguration::setURL(const std::string & url_)
LOG_TRACE(getLogger("StorageHDFSConfiguration"), "Using URL: {}, path: {}", url, path);
}
void StorageHDFSConfiguration::addStructureAndFormatToArgs(
void StorageHDFSConfiguration::addStructureAndFormatToArgsIfNeeded(
ASTs & args,
const String & structure_,
const String & format_,
ContextPtr context)
{
if (tryGetNamedCollectionWithOverrides(args, context))
if (auto collection = tryGetNamedCollectionWithOverrides(args, context))
{
/// In case of named collection, just add key-value pair "structure='...'"
/// at the end of arguments to override existed structure.
ASTs equal_func_args = {std::make_shared<ASTIdentifier>("structure"), std::make_shared<ASTLiteral>(structure_)};
auto equal_func = makeASTFunction("equals", std::move(equal_func_args));
args.push_back(equal_func);
/// In case of named collection, just add key-value pairs "format='...', structure='...'"
/// at the end of arguments to override existed format and structure with "auto" values.
if (collection->getOrDefault<String>("format", "auto") == "auto")
{
ASTs format_equal_func_args = {std::make_shared<ASTIdentifier>("format"), std::make_shared<ASTLiteral>(format_)};
auto format_equal_func = makeASTFunction("equals", std::move(format_equal_func_args));
args.push_back(format_equal_func);
}
if (collection->getOrDefault<String>("structure", "auto") == "auto")
{
ASTs structure_equal_func_args = {std::make_shared<ASTIdentifier>("structure"), std::make_shared<ASTLiteral>(structure_)};
auto structure_equal_func = makeASTFunction("equals", std::move(structure_equal_func_args));
args.push_back(structure_equal_func);
}
}
else
{
size_t count = args.size();
if (count == 0 || count > 4)
{
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Expected 1 to 4 arguments in table function, got {}", count);
}
if (count == 0 || count > getMaxNumberOfArguments())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 1 to {} arguments in table function hdfs, got {}", getMaxNumberOfArguments(), count);
auto format_literal = std::make_shared<ASTLiteral>(format_);
auto structure_literal = std::make_shared<ASTLiteral>(structure_);

View File

@ -16,6 +16,20 @@ public:
static constexpr auto type_name = "hdfs";
static constexpr auto engine_name = "HDFS";
/// All possible signatures for HDFS engine with structure argument (for example for hdfs table function).
static constexpr auto max_number_of_arguments_with_structure = 4;
static constexpr auto signatures_with_structure =
" - uri\n"
" - uri, format\n"
" - uri, format, structure\n"
" - uri, format, structure, compression_method\n";
/// All possible signatures for HDFS engine without structure argument (for example for HS table engine).
static constexpr auto max_number_of_arguments_without_structure = 3;
static constexpr auto signatures_without_structure =
" - uri\n"
" - uri, format\n"
" - uri, format, compression_method\n";
StorageHDFSConfiguration() = default;
StorageHDFSConfiguration(const StorageHDFSConfiguration & other);
@ -23,6 +37,9 @@ public:
std::string getTypeName() const override { return type_name; }
std::string getEngineName() const override { return engine_name; }
std::string getSignatures(bool with_structure = true) const { return with_structure ? signatures_with_structure : signatures_without_structure; }
size_t getMaxNumberOfArguments(bool with_structure = true) const { return with_structure ? max_number_of_arguments_with_structure : max_number_of_arguments_without_structure; }
Path getPath() const override { return path; }
void setPath(const Path & path_) override { path = path_; }
@ -39,7 +56,7 @@ public:
ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly) override;
void addStructureAndFormatToArgs(
void addStructureAndFormatToArgsIfNeeded(
ASTs & args,
const String & structure_,
const String & format_,

View File

@ -26,11 +26,11 @@ void StorageLocalConfiguration::fromNamedCollection(const NamedCollection & coll
void StorageLocalConfiguration::fromAST(ASTs & args, ContextPtr context, bool with_structure)
{
const size_t max_args_num = with_structure ? 4 : 3;
if (args.empty() || args.size() > max_args_num)
{
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Expected not more than {} arguments", max_args_num);
}
if (args.empty() || args.size() > getMaxNumberOfArguments(with_structure))
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Storage Local requires 1 to {} arguments. All supported signatures:\n{}",
getMaxNumberOfArguments(with_structure),
getSignatures(with_structure));
for (auto & arg : args)
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);

View File

@ -19,6 +19,20 @@ public:
using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr;
static constexpr auto type_name = "local";
/// All possible signatures for Local engine with structure argument (for example for local table function).
static constexpr auto max_number_of_arguments_with_structure = 4;
static constexpr auto signatures_with_structure =
" - path\n"
" - path, format\n"
" - path, format, structure\n"
" - path, format, structure, compression_method\n";
/// All possible signatures for S3 engine without structure argument (for example for Local table engine).
static constexpr auto max_number_of_arguments_without_structure = 3;
static constexpr auto signatures_without_structure =
" - path\n"
" - path, format\n"
" - path, format, compression_method\n";
StorageLocalConfiguration() = default;
StorageLocalConfiguration(const StorageLocalConfiguration & other) = default;
@ -26,6 +40,9 @@ public:
std::string getTypeName() const override { return type_name; }
std::string getEngineName() const override { return "Local"; }
std::string getSignatures(bool with_structure = true) const { return with_structure ? signatures_with_structure : signatures_without_structure; }
size_t getMaxNumberOfArguments(bool with_structure = true) const { return with_structure ? max_number_of_arguments_with_structure : max_number_of_arguments_without_structure; }
Path getPath() const override { return path; }
void setPath(const Path & path_) override { path = path_; }
@ -40,7 +57,7 @@ public:
ObjectStoragePtr createObjectStorage(ContextPtr, bool) override { return std::make_shared<LocalObjectStorage>("/"); }
void addStructureAndFormatToArgs(ASTs &, const String &, const String &, ContextPtr) override { }
void addStructureAndFormatToArgsIfNeeded(ASTs &, const String &, const String &, ContextPtr) override { }
private:
void fromNamedCollection(const NamedCollection & collection, ContextPtr context) override;

View File

@ -170,21 +170,20 @@ void StorageS3Configuration::fromNamedCollection(const NamedCollection & collect
void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_structure)
{
/// Supported signatures: S3('url') S3('url', 'format') S3('url', 'format', 'compression') S3('url', NOSIGN) S3('url', NOSIGN, 'format') S3('url', NOSIGN, 'format', 'compression') S3('url', 'aws_access_key_id', 'aws_secret_access_key') S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'session_token') S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format') S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'session_token', 'format') S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression')
/// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'session_token', 'format', 'compression')
/// with optional headers() function
size_t count = StorageURL::evalArgsAndCollectHeaders(args, headers_from_ast, context);
if (count == 0 || count > (with_structure ? 7 : 6))
if (count == 0 || count > getMaxNumberOfArguments(with_structure))
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Storage S3 requires 1 to 5 arguments: "
"url, [NOSIGN | access_key_id, secret_access_key], name of used format and [compression_method]");
"Storage S3 requires 1 to {} arguments. All supported signatures:\n{}",
getMaxNumberOfArguments(with_structure),
getSignatures(with_structure));
std::unordered_map<std::string_view, size_t> engine_args_to_idx;
bool no_sign_request = false;
/// For 2 arguments we support 2 possible variants:
/// When adding new arguments in the signature don't forget to update addStructureAndFormatToArgsIfNeeded as well.
/// For 2 arguments we support:
/// - s3(source, format)
/// - s3(source, NOSIGN)
/// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or not.
@ -196,10 +195,15 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_
else
engine_args_to_idx = {{"format", 1}};
}
/// For 3 arguments we support 2 possible variants:
/// For 3 arguments we support:
/// if with_structure == 0:
/// - s3(source, NOSIGN, format)
/// - s3(source, format, compression_method)
/// - s3(source, access_key_id, secret_access_key)
/// if with_structure == 1:
/// - s3(source, NOSIGN, format)
/// - s3(source, format, structure)
/// - s3(source, access_key_id, secret_access_key)
/// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or format name.
else if (count == 3)
{
@ -219,7 +223,7 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_
else
engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}};
}
/// For 4 arguments we support 3 possible variants:
/// For 4 arguments we support:
/// if with_structure == 0:
/// - s3(source, access_key_id, secret_access_key, session_token)
/// - s3(source, access_key_id, secret_access_key, format)
@ -229,7 +233,7 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_
/// - s3(source, access_key_id, secret_access_key, format),
/// - s3(source, access_key_id, secret_access_key, session_token)
/// - s3(source, NOSIGN, format, structure)
/// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN or not.
/// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN, format name of something else.
else if (count == 4)
{
auto second_arg = checkAndGetLiteralArgument<String>(args[1], "access_key_id/NOSIGN");
@ -258,7 +262,7 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_
}
}
}
/// For 5 arguments we support 2 possible variants:
/// For 5 arguments we support:
/// if with_structure == 0:
/// - s3(source, access_key_id, secret_access_key, session_token, format)
/// - s3(source, access_key_id, secret_access_key, format, compression)
@ -302,13 +306,16 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_
}
}
}
/// For 6 arguments we support:
/// if with_structure == 0:
/// - s3(source, access_key_id, secret_access_key, session_token, format, compression_method)
/// if with_structure == 1:
/// - s3(source, access_key_id, secret_access_key, format, structure, compression_method)
/// - s3(source, access_key_id, secret_access_key, session_token, format, structure)
else if (count == 6)
{
if (with_structure)
{
/// - s3(source, access_key_id, secret_access_key, format, structure, compression_method)
/// - s3(source, access_key_id, secret_access_key, session_token, format, structure)
/// We can distinguish them by looking at the 4-th argument: check if it's a format name or not
auto fourth_arg = checkAndGetLiteralArgument<String>(args[3], "format/session_token");
if (fourth_arg == "auto" || FormatFactory::instance().exists(fourth_arg))
{
@ -324,6 +331,7 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_
engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}, {"compression_method", 5}};
}
}
/// s3(source, access_key_id, secret_access_key, session_token, format, structure, compression_method)
else if (with_structure && count == 7)
{
engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}, {"structure", 5}, {"compression_method", 6}};
@ -365,24 +373,33 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_
keys = {url.key};
}
void StorageS3Configuration::addStructureAndFormatToArgs(
void StorageS3Configuration::addStructureAndFormatToArgsIfNeeded(
ASTs & args, const String & structure_, const String & format_, ContextPtr context)
{
if (tryGetNamedCollectionWithOverrides(args, context))
if (auto collection = tryGetNamedCollectionWithOverrides(args, context))
{
/// In case of named collection, just add key-value pair "structure='...'"
/// at the end of arguments to override existed structure.
ASTs equal_func_args = {std::make_shared<ASTIdentifier>("structure"), std::make_shared<ASTLiteral>(structure_)};
auto equal_func = makeASTFunction("equals", std::move(equal_func_args));
args.push_back(equal_func);
/// In case of named collection, just add key-value pairs "format='...', structure='...'"
/// at the end of arguments to override existed format and structure with "auto" values.
if (collection->getOrDefault<String>("format", "auto") == "auto")
{
ASTs format_equal_func_args = {std::make_shared<ASTIdentifier>("format"), std::make_shared<ASTLiteral>(format_)};
auto format_equal_func = makeASTFunction("equals", std::move(format_equal_func_args));
args.push_back(format_equal_func);
}
if (collection->getOrDefault<String>("structure", "auto") == "auto")
{
ASTs structure_equal_func_args = {std::make_shared<ASTIdentifier>("structure"), std::make_shared<ASTLiteral>(structure_)};
auto structure_equal_func = makeASTFunction("equals", std::move(structure_equal_func_args));
args.push_back(structure_equal_func);
}
}
else
{
HTTPHeaderEntries tmp_headers;
size_t count = StorageURL::evalArgsAndCollectHeaders(args, tmp_headers, context);
if (count == 0 || count > 6)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 1 to 6 arguments in table function, got {}", count);
if (count == 0 || count > getMaxNumberOfArguments())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 1 to {} arguments in table function s3, got {}", getMaxNumberOfArguments(), count);
auto format_literal = std::make_shared<ASTLiteral>(format_);
auto structure_literal = std::make_shared<ASTLiteral>(structure_);
@ -394,14 +411,18 @@ void StorageS3Configuration::addStructureAndFormatToArgs(
args.push_back(std::make_shared<ASTLiteral>("auto"));
args.push_back(structure_literal);
}
/// s3(s3_url, format) or s3(s3_url, NOSIGN)
/// s3(s3_url, format) or
/// s3(s3_url, NOSIGN)
/// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or not.
else if (count == 2)
{
auto second_arg = checkAndGetLiteralArgument<String>(args[1], "format/NOSIGN");
/// If there is NOSIGN, add format=auto before structure.
if (boost::iequals(second_arg, "NOSIGN"))
args.push_back(std::make_shared<ASTLiteral>("auto"));
args.push_back(format_literal);
else if (checkAndGetLiteralArgument<String>(args[1], "format") == "auto")
args[1] = format_literal;
args.push_back(structure_literal);
}
/// s3(source, format, structure) or
@ -413,21 +434,27 @@ void StorageS3Configuration::addStructureAndFormatToArgs(
auto second_arg = checkAndGetLiteralArgument<String>(args[1], "format/NOSIGN");
if (boost::iequals(second_arg, "NOSIGN"))
{
if (checkAndGetLiteralArgument<String>(args[2], "format") == "auto")
args[2] = format_literal;
args.push_back(structure_literal);
}
else if (second_arg == "auto" || FormatFactory::instance().exists(second_arg))
{
args[count - 1] = structure_literal;
if (second_arg == "auto")
args[1] = format_literal;
if (checkAndGetLiteralArgument<String>(args[2], "structure") == "auto")
args[2] = structure_literal;
}
else
{
/// Add format=auto before structure argument.
args.push_back(std::make_shared<ASTLiteral>("auto"));
/// Add format and structure arguments.
args.push_back(format_literal);
args.push_back(structure_literal);
}
}
/// s3(source, format, structure, compression_method) or
/// s3(source, access_key_id, secret_access_key, format) or
/// s3(source, access_key_id, secret_access_key, session_token) or
/// s3(source, NOSIGN, format, structure)
/// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN, format name or neither.
else if (count == 4)
@ -435,36 +462,93 @@ void StorageS3Configuration::addStructureAndFormatToArgs(
auto second_arg = checkAndGetLiteralArgument<String>(args[1], "format/NOSIGN");
if (boost::iequals(second_arg, "NOSIGN"))
{
args[count - 1] = structure_literal;
if (checkAndGetLiteralArgument<String>(args[2], "format") == "auto")
args[2] = format_literal;
if (checkAndGetLiteralArgument<String>(args[3], "structure") == "auto")
args[3] = structure_literal;
}
else if (second_arg == "auto" || FormatFactory::instance().exists(second_arg))
{
args[count - 2] = structure_literal;
if (second_arg == "auto")
args[1] = format_literal;
if (checkAndGetLiteralArgument<String>(args[2], "structure") == "auto")
args[2] = structure_literal;
}
else
{
args.push_back(structure_literal);
auto fourth_arg = checkAndGetLiteralArgument<String>(args[3], "format/session_token");
if (fourth_arg == "auto" || FormatFactory::instance().exists(fourth_arg))
{
if (checkAndGetLiteralArgument<String>(args[3], "format") == "auto")
args[3] = format_literal;
args.push_back(structure_literal);
}
else
{
args.push_back(format_literal);
args.push_back(structure_literal);
}
}
}
/// s3(source, access_key_id, secret_access_key, format, structure) or
/// s3(source, access_key_id, secret_access_key, session_token, format) or
/// s3(source, NOSIGN, format, structure, compression_method)
/// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN keyword name or not.
else if (count == 5)
{
auto sedond_arg = checkAndGetLiteralArgument<String>(args[1], "format/NOSIGN");
if (boost::iequals(sedond_arg, "NOSIGN"))
auto second_arg = checkAndGetLiteralArgument<String>(args[1], "format/NOSIGN");
if (boost::iequals(second_arg, "NOSIGN"))
{
args[count - 2] = structure_literal;
if (checkAndGetLiteralArgument<String>(args[2], "format") == "auto")
args[2] = format_literal;
if (checkAndGetLiteralArgument<String>(args[2], "structure") == "auto")
args[3] = structure_literal;
}
else
{
args[count - 1] = structure_literal;
auto fourth_arg = checkAndGetLiteralArgument<String>(args[3], "format/session_token");
if (fourth_arg == "auto" || FormatFactory::instance().exists(fourth_arg))
{
if (checkAndGetLiteralArgument<String>(args[3], "format") == "auto")
args[3] = format_literal;
if (checkAndGetLiteralArgument<String>(args[4], "structure") == "auto")
args[4] = structure_literal;
}
else
{
if (checkAndGetLiteralArgument<String>(args[4], "format") == "auto")
args[4] = format_literal;
args.push_back(structure_literal);
}
}
}
/// s3(source, access_key_id, secret_access_key, format, structure, compression)
/// s3(source, access_key_id, secret_access_key, format, structure, compression) or
/// s3(source, access_key_id, secret_access_key, session_token, format, structure)
else if (count == 6)
{
args[count - 2] = structure_literal;
auto fourth_arg = checkAndGetLiteralArgument<String>(args[3], "format/session_token");
if (fourth_arg == "auto" || FormatFactory::instance().exists(fourth_arg))
{
if (checkAndGetLiteralArgument<String>(args[3], "format") == "auto")
args[3] = format_literal;
if (checkAndGetLiteralArgument<String>(args[4], "structure") == "auto")
args[4] = structure_literal;
}
else
{
if (checkAndGetLiteralArgument<String>(args[4], "format") == "auto")
args[4] = format_literal;
if (checkAndGetLiteralArgument<String>(args[5], "format") == "auto")
args[5] = structure_literal;
}
}
/// s3(source, access_key_id, secret_access_key, session_token, format, structure, compression_method)
else if (count == 7)
{
if (checkAndGetLiteralArgument<String>(args[4], "format") == "auto")
args[4] = format_literal;
if (checkAndGetLiteralArgument<String>(args[5], "format") == "auto")
args[5] = structure_literal;
}
}
}

View File

@ -16,6 +16,43 @@ public:
static constexpr auto type_name = "s3";
static constexpr auto namespace_name = "bucket";
/// All possible signatures for S3 storage with structure argument (for example for s3 table function).
static constexpr auto max_number_of_arguments_with_structure = 7;
static constexpr auto signatures_with_structure =
" - url\n"
" - url, NOSIGN\n"
" - url, format\n"
" - url, NOSIGN, format\n"
" - url, format, structure\n"
" - url, NOSIGN, format, structure\n"
" - url, format, structure, compression_method\n"
" - url, NOSIGN, format, structure, compression_method\n"
" - url, access_key_id, secret_access_key\n"
" - url, access_key_id, secret_access_key, session_token\n"
" - url, access_key_id, secret_access_key, format\n"
" - url, access_key_id, secret_access_key, session_token, format\n"
" - url, access_key_id, secret_access_key, format, structure\n"
" - url, access_key_id, secret_access_key, session_token, format, structure\n"
" - url, access_key_id, secret_access_key, format, structure, compression_method\n"
" - url, access_key_id, secret_access_key, session_token, format, structure, compression_method\n"
"All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)";
/// All possible signatures for S3 storage without structure argument (for example for S3 table engine).
static constexpr auto max_number_of_arguments_without_structure = 6;
static constexpr auto signatures_without_structure =
" - url\n"
" - url, NOSIGN\n"
" - url, format\n"
" - url, NOSIGN, format\n"
" - url, format, compression_method\n"
" - url, NOSIGN, format, compression_method\n"
" - url, access_key_id, secret_access_key\n"
" - url, access_key_id, secret_access_key, session_token\n"
" - url, access_key_id, secret_access_key, format\n"
" - url, access_key_id, secret_access_key, session_token, format\n"
" - url, access_key_id, secret_access_key, format, compression_method\n"
" - url, access_key_id, secret_access_key, session_token, format, compression_method\n"
"All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)";
StorageS3Configuration() = default;
StorageS3Configuration(const StorageS3Configuration & other);
@ -24,6 +61,9 @@ public:
std::string getEngineName() const override { return url.storage_name; }
std::string getNamespaceType() const override { return namespace_name; }
std::string getSignatures(bool with_structure = true) const { return with_structure ? signatures_with_structure : signatures_without_structure; }
size_t getMaxNumberOfArguments(bool with_structure = true) const { return with_structure ? max_number_of_arguments_with_structure : max_number_of_arguments_without_structure; }
Path getPath() const override { return url.key; }
void setPath(const Path & path) override { url.key = path; }
@ -44,7 +84,7 @@ public:
ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly) override;
void addStructureAndFormatToArgs(
void addStructureAndFormatToArgsIfNeeded(
ASTs & args,
const String & structure,
const String & format,

View File

@ -180,7 +180,9 @@ public:
virtual String getNamespace() const = 0;
virtual StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const = 0;
virtual void addStructureAndFormatToArgs(
/// Add/replace structure and format arguments in the AST arguments if they have 'auto' values.
virtual void addStructureAndFormatToArgsIfNeeded(
ASTs & args, const String & structure_, const String & format_, ContextPtr context) = 0;
bool withPartitionWildcard() const;

View File

@ -103,7 +103,7 @@ void StorageObjectStorageCluster::updateQueryToSendIfNeeded(
ASTPtr cluster_name_arg = args.front();
args.erase(args.begin());
configuration->addStructureAndFormatToArgs(args, structure, configuration->format, context);
configuration->addStructureAndFormatToArgsIfNeeded(args, structure, configuration->format, context);
args.insert(args.begin(), cluster_name_arg);
}

View File

@ -33,7 +33,6 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
extern const int REPLICA_ALREADY_EXISTS;
extern const int INCOMPATIBLE_COLUMNS;
}
namespace
@ -108,8 +107,12 @@ private:
}
};
ObjectStorageQueueMetadata::ObjectStorageQueueMetadata(const fs::path & zookeeper_path_, const ObjectStorageQueueSettings & settings_)
ObjectStorageQueueMetadata::ObjectStorageQueueMetadata(
const fs::path & zookeeper_path_,
const ObjectStorageQueueTableMetadata & table_metadata_,
const ObjectStorageQueueSettings & settings_)
: settings(settings_)
, table_metadata(table_metadata_)
, zookeeper_path(zookeeper_path_)
, buckets_num(getBucketsNum(settings_))
, log(getLogger("StorageObjectStorageQueue(" + zookeeper_path_.string() + ")"))
@ -144,11 +147,6 @@ void ObjectStorageQueueMetadata::shutdown()
task->deactivate();
}
void ObjectStorageQueueMetadata::checkSettings(const ObjectStorageQueueSettings & settings_) const
{
ObjectStorageQueueTableMetadata::checkEquals(settings, settings_);
}
ObjectStorageQueueMetadata::FileStatusPtr ObjectStorageQueueMetadata::getFileStatus(const std::string & path)
{
return local_file_statuses->get(path, /* create */false);
@ -219,13 +217,14 @@ ObjectStorageQueueMetadata::tryAcquireBucket(const Bucket & bucket, const Proces
return ObjectStorageQueueOrderedFileMetadata::tryAcquireBucket(zookeeper_path, bucket, processor, log);
}
void ObjectStorageQueueMetadata::initialize(
const ConfigurationPtr & configuration,
const StorageInMemoryMetadata & storage_metadata)
void ObjectStorageQueueMetadata::syncWithKeeper(
const fs::path & zookeeper_path,
const ObjectStorageQueueTableMetadata & table_metadata,
const ObjectStorageQueueSettings & settings,
LoggerPtr log)
{
const auto metadata_from_table = ObjectStorageQueueTableMetadata(*configuration, settings, storage_metadata);
const auto & columns_from_table = storage_metadata.getColumns();
const auto table_metadata_path = zookeeper_path / "metadata";
const auto buckets_num = getBucketsNum(settings);
const auto metadata_paths = settings.mode == ObjectStorageQueueMode::ORDERED
? ObjectStorageQueueOrderedFileMetadata::getMetadataPaths(buckets_num)
: ObjectStorageQueueUnorderedFileMetadata::getMetadataPaths();
@ -237,24 +236,19 @@ void ObjectStorageQueueMetadata::initialize(
{
if (zookeeper->exists(table_metadata_path))
{
const auto metadata_from_zk = ObjectStorageQueueTableMetadata::parse(zookeeper->get(fs::path(zookeeper_path) / "metadata"));
const auto columns_from_zk = ColumnsDescription::parse(metadata_from_zk.columns);
const auto metadata_str = zookeeper->get(fs::path(zookeeper_path) / "metadata");
const auto metadata_from_zk = ObjectStorageQueueTableMetadata::parse(metadata_str);
metadata_from_table.checkEquals(metadata_from_zk);
if (columns_from_zk != columns_from_table)
{
throw Exception(
ErrorCodes::INCOMPATIBLE_COLUMNS,
"Table columns structure in ZooKeeper is different from local table structure. "
"Local columns:\n{}\nZookeeper columns:\n{}",
columns_from_table.toString(), columns_from_zk.toString());
}
LOG_TRACE(log, "Metadata in keeper: {}", metadata_str);
table_metadata.checkEquals(metadata_from_zk);
return;
}
Coordination::Requests requests;
requests.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent));
requests.emplace_back(zkutil::makeCreateRequest(table_metadata_path, metadata_from_table.toString(), zkutil::CreateMode::Persistent));
requests.emplace_back(zkutil::makeCreateRequest(
table_metadata_path, table_metadata.toString(), zkutil::CreateMode::Persistent));
for (const auto & path : metadata_paths)
{
@ -263,16 +257,27 @@ void ObjectStorageQueueMetadata::initialize(
}
if (!settings.last_processed_path.value.empty())
getFileMetadata(settings.last_processed_path)->setProcessedAtStartRequests(requests, zookeeper);
{
ObjectStorageQueueOrderedFileMetadata(
zookeeper_path,
settings.last_processed_path,
std::make_shared<FileStatus>(),
/* bucket_info */nullptr,
buckets_num,
settings.loading_retries,
log).setProcessedAtStartRequests(requests, zookeeper);
}
Coordination::Responses responses;
auto code = zookeeper->tryMulti(requests, responses);
if (code == Coordination::Error::ZNODEEXISTS)
{
auto exception = zkutil::KeeperMultiException(code, requests, responses);
LOG_INFO(log, "Got code `{}` for path: {}. "
"It looks like the table {} was created by another server at the same moment, "
"will retry", code, exception.getPathForFirstFailedOp(), zookeeper_path.string());
"will retry",
code, exception.getPathForFirstFailedOp(), zookeeper_path.string());
continue;
}
else if (code != Coordination::Error::ZOK)

View File

@ -8,6 +8,7 @@
#include <Storages/ObjectStorageQueue/ObjectStorageQueueIFileMetadata.h>
#include <Storages/ObjectStorageQueue/ObjectStorageQueueOrderedFileMetadata.h>
#include <Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h>
#include <Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h>
#include <Common/ZooKeeper/ZooKeeper.h>
namespace fs = std::filesystem;
@ -52,11 +53,19 @@ public:
using Bucket = size_t;
using Processor = std::string;
ObjectStorageQueueMetadata(const fs::path & zookeeper_path_, const ObjectStorageQueueSettings & settings_);
ObjectStorageQueueMetadata(
const fs::path & zookeeper_path_,
const ObjectStorageQueueTableMetadata & table_metadata_,
const ObjectStorageQueueSettings & settings_);
~ObjectStorageQueueMetadata();
void initialize(const ConfigurationPtr & configuration, const StorageInMemoryMetadata & storage_metadata);
void checkSettings(const ObjectStorageQueueSettings & settings) const;
static void syncWithKeeper(
const fs::path & zookeeper_path,
const ObjectStorageQueueTableMetadata & table_metadata,
const ObjectStorageQueueSettings & settings,
LoggerPtr log);
void shutdown();
FileMetadataPtr getFileMetadata(const std::string & path, ObjectStorageQueueOrderedFileMetadata::BucketInfoPtr bucket_info = {});
@ -72,11 +81,17 @@ public:
static size_t getBucketsNum(const ObjectStorageQueueSettings & settings);
static size_t getBucketsNum(const ObjectStorageQueueTableMetadata & settings);
void checkTableMetadataEquals(const ObjectStorageQueueMetadata & other);
const ObjectStorageQueueTableMetadata & getTableMetadata() const { return table_metadata; }
ObjectStorageQueueTableMetadata & getTableMetadata() { return table_metadata; }
private:
void cleanupThreadFunc();
void cleanupThreadFuncImpl();
const ObjectStorageQueueSettings settings;
ObjectStorageQueueSettings settings;
ObjectStorageQueueTableMetadata table_metadata;
const fs::path zookeeper_path;
const size_t buckets_num;
@ -89,4 +104,6 @@ private:
std::shared_ptr<LocalFileStatuses> local_file_statuses;
};
using ObjectStorageQueueMetadataPtr = std::unique_ptr<ObjectStorageQueueMetadata>;
}

View File

@ -14,19 +14,23 @@ ObjectStorageQueueMetadataFactory & ObjectStorageQueueMetadataFactory::instance(
return ret;
}
ObjectStorageQueueMetadataFactory::FilesMetadataPtr
ObjectStorageQueueMetadataFactory::getOrCreate(const std::string & zookeeper_path, const ObjectStorageQueueSettings & settings)
ObjectStorageQueueMetadataFactory::FilesMetadataPtr ObjectStorageQueueMetadataFactory::getOrCreate(
const std::string & zookeeper_path,
ObjectStorageQueueMetadataPtr metadata)
{
std::lock_guard lock(mutex);
auto it = metadata_by_path.find(zookeeper_path);
if (it == metadata_by_path.end())
{
auto files_metadata = std::make_shared<ObjectStorageQueueMetadata>(zookeeper_path, settings);
it = metadata_by_path.emplace(zookeeper_path, std::move(files_metadata)).first;
it = metadata_by_path.emplace(zookeeper_path, std::move(metadata)).first;
}
else
{
it->second.metadata->checkSettings(settings);
auto & metadata_from_table = metadata->getTableMetadata();
auto & metadata_from_keeper = it->second.metadata->getTableMetadata();
metadata_from_table.checkEquals(metadata_from_keeper);
it->second.ref_count += 1;
}
return it->second.metadata;

View File

@ -13,7 +13,9 @@ public:
static ObjectStorageQueueMetadataFactory & instance();
FilesMetadataPtr getOrCreate(const std::string & zookeeper_path, const ObjectStorageQueueSettings & settings);
FilesMetadataPtr getOrCreate(
const std::string & zookeeper_path,
ObjectStorageQueueMetadataPtr metadata);
void remove(const std::string & zookeeper_path);

View File

@ -1,6 +1,5 @@
#include <config.h>
#include <Poco/JSON/JSON.h>
#include <Poco/JSON/Object.h>
#include <Poco/JSON/Parser.h>
#include <Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h>
@ -32,18 +31,18 @@ namespace
ObjectStorageQueueTableMetadata::ObjectStorageQueueTableMetadata(
const StorageObjectStorage::Configuration & configuration,
const ObjectStorageQueueSettings & engine_settings,
const StorageInMemoryMetadata & storage_metadata)
const ColumnsDescription & columns_,
const std::string & format_)
: format_name(format_)
, columns(columns_.toString())
, after_processing(engine_settings.after_processing.toString())
, mode(engine_settings.mode.toString())
, tracked_files_limit(engine_settings.tracked_files_limit)
, tracked_file_ttl_sec(engine_settings.tracked_file_ttl_sec)
, buckets(engine_settings.buckets)
, processing_threads_num(engine_settings.processing_threads_num)
{
format_name = configuration.format;
after_processing = engine_settings.after_processing.toString();
mode = engine_settings.mode.toString();
tracked_files_limit = engine_settings.tracked_files_limit;
tracked_file_ttl_sec = engine_settings.tracked_file_ttl_sec;
buckets = engine_settings.buckets;
processing_threads_num = engine_settings.processing_threads_num;
columns = storage_metadata.getColumns().toString();
}
String ObjectStorageQueueTableMetadata::toString() const
@ -65,48 +64,40 @@ String ObjectStorageQueueTableMetadata::toString() const
return oss.str();
}
void ObjectStorageQueueTableMetadata::read(const String & metadata_str)
template <typename T>
static auto getOrDefault(
const Poco::JSON::Object::Ptr & json,
const std::string & setting,
const std::string & compatibility_prefix,
const T & default_value)
{
Poco::JSON::Parser parser;
auto json = parser.parse(metadata_str).extract<Poco::JSON::Object::Ptr>();
if (!compatibility_prefix.empty() && json->has(compatibility_prefix + setting))
return json->getValue<T>(compatibility_prefix + setting);
after_processing = json->getValue<String>("after_processing");
mode = json->getValue<String>("mode");
if (json->has(setting))
return json->getValue<T>(setting);
format_name = json->getValue<String>("format_name");
columns = json->getValue<String>("columns");
return default_value;
}
/// Check with "s3queue_" prefix for compatibility.
{
if (json->has("s3queue_tracked_files_limit"))
tracked_files_limit = json->getValue<UInt64>("s3queue_tracked_files_limit");
if (json->has("s3queue_tracked_file_ttl_sec"))
tracked_file_ttl_sec = json->getValue<UInt64>("s3queue_tracked_file_ttl_sec");
if (json->has("s3queue_processing_threads_num"))
processing_threads_num = json->getValue<UInt64>("s3queue_processing_threads_num");
}
if (json->has("tracked_files_limit"))
tracked_files_limit = json->getValue<UInt64>("tracked_files_limit");
if (json->has("tracked_file_ttl_sec"))
tracked_file_ttl_sec = json->getValue<UInt64>("tracked_file_ttl_sec");
if (json->has("last_processed_file"))
last_processed_path = json->getValue<String>("last_processed_file");
if (json->has("processing_threads_num"))
processing_threads_num = json->getValue<UInt64>("processing_threads_num");
if (json->has("buckets"))
buckets = json->getValue<UInt64>("buckets");
ObjectStorageQueueTableMetadata::ObjectStorageQueueTableMetadata(const Poco::JSON::Object::Ptr & json)
: format_name(json->getValue<String>("format_name"))
, columns(json->getValue<String>("columns"))
, after_processing(json->getValue<String>("after_processing"))
, mode(json->getValue<String>("mode"))
, tracked_files_limit(getOrDefault(json, "tracked_files_limit", "s3queue_", 0))
, tracked_file_ttl_sec(getOrDefault(json, "tracked_files_ttl_sec", "s3queue_", 0))
, buckets(getOrDefault(json, "buckets", "", 0))
, processing_threads_num(getOrDefault(json, "processing_threads_num", "s3queue_", 1))
, last_processed_path(getOrDefault<String>(json, "last_processed_file", "s3queue_", ""))
{
}
ObjectStorageQueueTableMetadata ObjectStorageQueueTableMetadata::parse(const String & metadata_str)
{
ObjectStorageQueueTableMetadata metadata;
metadata.read(metadata_str);
return metadata;
Poco::JSON::Parser parser;
auto json = parser.parse(metadata_str).extract<Poco::JSON::Object::Ptr>();
return ObjectStorageQueueTableMetadata(json);
}
void ObjectStorageQueueTableMetadata::checkEquals(const ObjectStorageQueueTableMetadata & from_zk) const
@ -181,72 +172,17 @@ void ObjectStorageQueueTableMetadata::checkImmutableFieldsEquals(const ObjectSto
ErrorCodes::METADATA_MISMATCH,
"Existing table metadata in ZooKeeper differs in processing buckets. "
"Stored in ZooKeeper: {}, local: {}",
ObjectStorageQueueMetadata::getBucketsNum(*this), ObjectStorageQueueMetadata::getBucketsNum(from_zk));
ObjectStorageQueueMetadata::getBucketsNum(from_zk), ObjectStorageQueueMetadata::getBucketsNum(*this));
}
}
if (columns != from_zk.columns)
throw Exception(
ErrorCodes::METADATA_MISMATCH,
"Existing table metadata in ZooKeeper differs in columns. "
"Stored in ZooKeeper: {}, local: {}",
from_zk.columns,
columns);
}
void ObjectStorageQueueTableMetadata::checkEquals(const ObjectStorageQueueSettings & current, const ObjectStorageQueueSettings & expected)
{
if (current.after_processing != expected.after_processing)
throw Exception(
ErrorCodes::METADATA_MISMATCH,
"Existing table metadata in ZooKeeper differs "
"in action after processing. Stored in ZooKeeper: {}, local: {}",
expected.after_processing.toString(),
current.after_processing.toString());
if (current.mode != expected.mode)
throw Exception(
ErrorCodes::METADATA_MISMATCH,
"Existing table metadata in ZooKeeper differs in engine mode. "
"Stored in ZooKeeper: {}, local: {}",
expected.mode.toString(),
current.mode.toString());
if (current.tracked_files_limit != expected.tracked_files_limit)
throw Exception(
ErrorCodes::METADATA_MISMATCH,
"Existing table metadata in ZooKeeper differs in max set size. "
"Stored in ZooKeeper: {}, local: {}",
expected.tracked_files_limit,
current.tracked_files_limit);
if (current.tracked_file_ttl_sec != expected.tracked_file_ttl_sec)
throw Exception(
ErrorCodes::METADATA_MISMATCH,
"Existing table metadata in ZooKeeper differs in max set age. "
"Stored in ZooKeeper: {}, local: {}",
expected.tracked_file_ttl_sec,
current.tracked_file_ttl_sec);
if (current.last_processed_path.value != expected.last_processed_path.value)
throw Exception(
ErrorCodes::METADATA_MISMATCH,
"Existing table metadata in ZooKeeper differs in last_processed_path. "
"Stored in ZooKeeper: {}, local: {}",
expected.last_processed_path.value,
current.last_processed_path.value);
if (current.mode == ObjectStorageQueueMode::ORDERED)
{
if (current.buckets != expected.buckets)
{
throw Exception(
ErrorCodes::METADATA_MISMATCH,
"Existing table metadata in ZooKeeper differs in buckets setting. "
"Stored in ZooKeeper: {}, local: {}",
expected.buckets, current.buckets);
}
if (ObjectStorageQueueMetadata::getBucketsNum(current) != ObjectStorageQueueMetadata::getBucketsNum(expected))
{
throw Exception(
ErrorCodes::METADATA_MISMATCH,
"Existing table metadata in ZooKeeper differs in processing buckets. "
"Stored in ZooKeeper: {}, local: {}",
ObjectStorageQueueMetadata::getBucketsNum(current), ObjectStorageQueueMetadata::getBucketsNum(expected));
}
}
}
}

View File

@ -3,6 +3,8 @@
#include <Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h>
#include <Storages/StorageInMemoryMetadata.h>
#include <Storages/ObjectStorage/StorageObjectStorage.h>
#include <Poco/JSON/JSON.h>
#include <Poco/JSON/Object.h>
#include <base/types.h>
namespace DB
@ -16,29 +18,28 @@ class ReadBuffer;
*/
struct ObjectStorageQueueTableMetadata
{
String format_name;
String columns;
String after_processing;
String mode;
UInt64 tracked_files_limit = 0;
UInt64 tracked_file_ttl_sec = 0;
UInt64 buckets = 0;
UInt64 processing_threads_num = 1;
String last_processed_path;
const String format_name;
const String columns;
const String after_processing;
const String mode;
const UInt64 tracked_files_limit;
const UInt64 tracked_file_ttl_sec;
const UInt64 buckets;
const UInt64 processing_threads_num;
const String last_processed_path;
ObjectStorageQueueTableMetadata() = default;
ObjectStorageQueueTableMetadata(
const StorageObjectStorage::Configuration & configuration,
const ObjectStorageQueueSettings & engine_settings,
const StorageInMemoryMetadata & storage_metadata);
const ColumnsDescription & columns_,
const std::string & format_);
explicit ObjectStorageQueueTableMetadata(const Poco::JSON::Object::Ptr & json);
void read(const String & metadata_str);
static ObjectStorageQueueTableMetadata parse(const String & metadata_str);
String toString() const;
void checkEquals(const ObjectStorageQueueTableMetadata & from_zk) const;
static void checkEquals(const ObjectStorageQueueSettings & current, const ObjectStorageQueueSettings & expected);
private:
void checkImmutableFieldsEquals(const ObjectStorageQueueTableMetadata & from_zk) const;

View File

@ -85,7 +85,10 @@ namespace
}
}
std::shared_ptr<ObjectStorageQueueLog> getQueueLog(const ObjectStoragePtr & storage, const ContextPtr & context, const ObjectStorageQueueSettings & table_settings)
std::shared_ptr<ObjectStorageQueueLog> getQueueLog(
const ObjectStoragePtr & storage,
const ContextPtr & context,
const ObjectStorageQueueSettings & table_settings)
{
const auto & settings = context->getSettingsRef();
switch (storage->getType())
@ -105,7 +108,6 @@ namespace
default:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected object storage type: {}", storage->getType());
}
}
}
@ -161,21 +163,14 @@ StorageObjectStorageQueue::StorageObjectStorageQueue(
setInMemoryMetadata(storage_metadata);
LOG_INFO(log, "Using zookeeper path: {}", zk_path.string());
task = getContext()->getSchedulePool().createTask("ObjectStorageQueueStreamingTask", [this] { threadFunc(); });
/// Get metadata manager from ObjectStorageQueueMetadataFactory,
/// it will increase the ref count for the metadata object.
/// The ref count is decreased when StorageObjectStorageQueue::drop() method is called.
files_metadata = ObjectStorageQueueMetadataFactory::instance().getOrCreate(zk_path, *queue_settings);
try
{
files_metadata->initialize(configuration_, storage_metadata);
}
catch (...)
{
ObjectStorageQueueMetadataFactory::instance().remove(zk_path);
throw;
}
ObjectStorageQueueTableMetadata table_metadata(*queue_settings, storage_metadata.getColumns(), configuration_->format);
ObjectStorageQueueMetadata::syncWithKeeper(zk_path, table_metadata, *queue_settings, log);
auto queue_metadata = std::make_unique<ObjectStorageQueueMetadata>(zk_path, std::move(table_metadata), *queue_settings);
files_metadata = ObjectStorageQueueMetadataFactory::instance().getOrCreate(zk_path, std::move(queue_metadata));
task = getContext()->getSchedulePool().createTask("ObjectStorageQueueStreamingTask", [this] { threadFunc(); });
}
void StorageObjectStorageQueue::startup()

View File

@ -23,7 +23,6 @@ class ITableFunctionCluster : public Base
{
public:
String getName() const override = 0;
String getSignature() const override = 0;
static void updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure_, const String & format_, const ContextPtr & context)
{
@ -46,7 +45,11 @@ protected:
void parseArgumentsImpl(ASTs & args, const ContextPtr & context) override
{
if (args.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "The signature of table function {} shall be the following:\n{}", getName(), getSignature());
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"The function {} should have arguments. The first argument must be the cluster name and the rest are the arguments of "
"corresponding table function",
getName());
/// Evaluate only first argument, everything else will be done Base class
args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(args[0], context);

View File

@ -57,7 +57,7 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, Context
void ITableFunctionFileLike::parseArgumentsImpl(ASTs & args, const ContextPtr & context)
{
if (args.empty() || args.size() > 4)
if (args.empty() || args.size() > getMaxNumberOfArguments())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "The signature of table function {} shall be the following:\n{}", getName(), getSignature());
for (auto & arg : args)

View File

@ -15,6 +15,7 @@ class Context;
class ITableFunctionFileLike : public ITableFunction
{
public:
static constexpr auto max_number_of_arguments = 4;
static constexpr auto signature = " - filename\n"
" - filename, format\n"
" - filename, format, structure\n"
@ -32,7 +33,7 @@ public:
NameSet getVirtualsToCheckBeforeUsingStructureHint() const override;
static size_t getMaxNumberOfArguments() { return 4; }
static size_t getMaxNumberOfArguments() { return max_number_of_arguments; }
static void updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure, const String & format, const ContextPtr &);

View File

@ -23,83 +23,42 @@ struct AzureDefinition
{
static constexpr auto name = "azureBlobStorage";
static constexpr auto storage_type_name = "Azure";
static constexpr auto signature = " - connection_string, container_name, blobpath\n"
" - connection_string, container_name, blobpath, structure \n"
" - connection_string, container_name, blobpath, format \n"
" - connection_string, container_name, blobpath, format, compression \n"
" - connection_string, container_name, blobpath, format, compression, structure \n"
" - storage_account_url, container_name, blobpath, account_name, account_key\n"
" - storage_account_url, container_name, blobpath, account_name, account_key, structure\n"
" - storage_account_url, container_name, blobpath, account_name, account_key, format\n"
" - storage_account_url, container_name, blobpath, account_name, account_key, format, compression\n"
" - storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure\n";
static constexpr auto max_number_of_arguments = 8;
};
struct S3Definition
{
static constexpr auto name = "s3";
static constexpr auto storage_type_name = "S3";
static constexpr auto signature = " - url\n"
" - url, format\n"
" - url, format, structure\n"
" - url, format, structure, compression_method\n"
" - url, access_key_id, secret_access_key\n"
" - url, access_key_id, secret_access_key, session_token\n"
" - url, access_key_id, secret_access_key, format\n"
" - url, access_key_id, secret_access_key, session_token, format\n"
" - url, access_key_id, secret_access_key, format, structure\n"
" - url, access_key_id, secret_access_key, session_token, format, structure\n"
" - url, access_key_id, secret_access_key, format, structure, compression_method\n"
" - url, access_key_id, secret_access_key, session_token, format, structure, compression_method\n"
"All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)";
static constexpr auto max_number_of_arguments = 8;
};
struct GCSDefinition
{
static constexpr auto name = "gcs";
static constexpr auto storage_type_name = "GCS";
static constexpr auto signature = S3Definition::signature;
static constexpr auto max_number_of_arguments = S3Definition::max_number_of_arguments;
};
struct COSNDefinition
{
static constexpr auto name = "cosn";
static constexpr auto storage_type_name = "COSN";
static constexpr auto signature = S3Definition::signature;
static constexpr auto max_number_of_arguments = S3Definition::max_number_of_arguments;
};
struct OSSDefinition
{
static constexpr auto name = "oss";
static constexpr auto storage_type_name = "OSS";
static constexpr auto signature = S3Definition::signature;
static constexpr auto max_number_of_arguments = S3Definition::max_number_of_arguments;
};
struct HDFSDefinition
{
static constexpr auto name = "hdfs";
static constexpr auto storage_type_name = "HDFS";
static constexpr auto signature = " - uri\n"
" - uri, format\n"
" - uri, format, structure\n"
" - uri, format, structure, compression_method\n";
static constexpr auto max_number_of_arguments = 4;
};
struct LocalDefinition
{
static constexpr auto name = "local";
static constexpr auto storage_type_name = "Local";
static constexpr auto signature = " - path\n"
" - path, format\n"
" - path, format, structure\n"
" - path, format, structure, compression_method\n";
static constexpr auto max_number_of_arguments = 4;
};
template <typename Definition, typename Configuration>
@ -107,14 +66,9 @@ class TableFunctionObjectStorage : public ITableFunction
{
public:
static constexpr auto name = Definition::name;
static constexpr auto signature = Definition::signature;
static size_t getMaxNumberOfArguments() { return Definition::max_number_of_arguments; }
String getName() const override { return name; }
virtual String getSignature() const { return signature; }
bool hasStaticStructure() const override { return configuration->structure != "auto"; }
bool needStructureHint() const override { return configuration->structure == "auto"; }
@ -142,7 +96,7 @@ public:
const String & format,
const ContextPtr & context)
{
Configuration().addStructureAndFormatToArgs(args, structure, format, context);
Configuration().addStructureAndFormatToArgsIfNeeded(args, structure, format, context);
}
protected:

View File

@ -19,40 +19,22 @@ struct AzureClusterDefinition
{
static constexpr auto name = "azureBlobStorageCluster";
static constexpr auto storage_type_name = "AzureBlobStorageCluster";
static constexpr auto signature = " - cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]";
static constexpr auto max_number_of_arguments = AzureDefinition::max_number_of_arguments + 1;
};
struct S3ClusterDefinition
{
static constexpr auto name = "s3Cluster";
static constexpr auto storage_type_name = "S3Cluster";
static constexpr auto signature = " - cluster, url\n"
" - cluster, url, format\n"
" - cluster, url, format, structure\n"
" - cluster, url, access_key_id, secret_access_key\n"
" - cluster, url, format, structure, compression_method\n"
" - cluster, url, access_key_id, secret_access_key, format\n"
" - cluster, url, access_key_id, secret_access_key, format, structure\n"
" - cluster, url, access_key_id, secret_access_key, format, structure, compression_method\n"
" - cluster, url, access_key_id, secret_access_key, session_token, format, structure, compression_method\n"
"All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)";
static constexpr auto max_number_of_arguments = S3Definition::max_number_of_arguments + 1;
};
struct HDFSClusterDefinition
{
static constexpr auto name = "hdfsCluster";
static constexpr auto storage_type_name = "HDFSCluster";
static constexpr auto signature = " - cluster_name, uri\n"
" - cluster_name, uri, format\n"
" - cluster_name, uri, format, structure\n"
" - cluster_name, uri, format, structure, compression_method\n";
static constexpr auto max_number_of_arguments = HDFSDefinition::max_number_of_arguments + 1;
};
/**
* Class implementing s3/hdfs/azureBlobStorage)Cluster(...) table functions,
* Class implementing s3/hdfs/azureBlobStorageCluster(...) table functions,
* which allow to process many files from S3/HDFS/Azure blob storage on a specific cluster.
* On initiator it creates a connection to _all_ nodes in cluster, discloses asterisks
* in file path and dispatch each file dynamically.
@ -64,10 +46,8 @@ class TableFunctionObjectStorageCluster : public ITableFunctionCluster<TableFunc
{
public:
static constexpr auto name = Definition::name;
static constexpr auto signature = Definition::signature;
String getName() const override { return name; }
String getSignature() const override { return signature; }
protected:
using Base = TableFunctionObjectStorage<Definition, Configuration>;

View File

@ -735,11 +735,14 @@ def test_mutation_with_broken_projection(cluster):
f"ALTER TABLE {table_name} DELETE WHERE _part == 'all_0_0_0_4' SETTINGS mutations_sync = 1"
)
parts = get_parts(node, table_name)
# All parts changes because this is how alter delete works,
# but all parts apart from the first have only hardlinks to files in previous part.
assert ["all_0_0_0_5", "all_1_1_0_5", "all_2_2_0_5", "all_3_3_0_5"] == get_parts(
node, table_name
) or ["all_1_1_0_5", "all_2_2_0_5", "all_3_3_0_5"] == get_parts(node, table_name)
assert ["all_0_0_0_5", "all_1_1_0_5", "all_2_2_0_5", "all_3_3_0_5"] == parts or [
"all_1_1_0_5",
"all_2_2_0_5",
"all_3_3_0_5",
] == parts
# Still broken because it was hardlinked.
broken = get_broken_projections_info(node, table_name)
@ -752,11 +755,13 @@ def test_mutation_with_broken_projection(cluster):
f"ALTER TABLE {table_name} DELETE WHERE c == 13 SETTINGS mutations_sync = 1"
)
assert ["all_1_1_0_6", "all_2_2_0_6", "all_3_3_0_6"] == get_parts(
node, table_name
) or ["all_0_0_0_6", "all_1_1_0_6", "all_2_2_0_6", "all_3_3_0_6"] == get_parts(
node, table_name
)
parts = get_parts(node, table_name)
assert ["all_1_1_0_6", "all_2_2_0_6", "all_3_3_0_6"] == parts or [
"all_0_0_0_6",
"all_1_1_0_6",
"all_2_2_0_6",
"all_3_3_0_6",
] == parts
# Not broken anymore.
assert not get_broken_projections_info(node, table_name)

View File

@ -70,6 +70,12 @@ def get_mysql_conn(cluster):
def fill_tables(cluster, dbname):
fill_nodes(nodes, dbname)
node1.query(
f"""CREATE TABLE {dbname}.example_s3_engine_table (name String, value UInt32)
ENGINE = S3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/test-data.csv.gz', 'CSV', 'gzip')
SETTINGS input_format_with_names_use_header = 0"""
)
conn = get_mysql_conn(cluster)
with conn.cursor() as cursor:
@ -136,6 +142,7 @@ def test_restore_table(start_cluster):
node2.query(f"BACKUP DATABASE replicated TO {backup_name}")
node2.query("DROP TABLE replicated.example_s3_engine_table")
node2.query("DROP TABLE replicated.mysql_schema_inference_engine")
node2.query("DROP TABLE replicated.mysql_schema_inference_function")
@ -149,6 +156,13 @@ def test_restore_table(start_cluster):
)
node1.query(f"SYSTEM SYNC DATABASE REPLICA replicated")
assert (
node1.query(
"SELECT engine FROM system.tables where database = 'replicated' and name = 'example_s3_engine_table'"
)
== "S3\n"
)
assert (
node1.query(
"SELECT count(), sum(id) FROM replicated.mysql_schema_inference_engine"
@ -175,6 +189,7 @@ def test_restore_table_null(start_cluster):
node2.query(f"BACKUP DATABASE replicated2 TO {backup_name}")
node2.query("DROP TABLE replicated2.example_s3_engine_table")
node2.query("DROP TABLE replicated2.mysql_schema_inference_engine")
node2.query("DROP TABLE replicated2.mysql_schema_inference_function")
@ -188,6 +203,13 @@ def test_restore_table_null(start_cluster):
)
node1.query(f"SYSTEM SYNC DATABASE REPLICA replicated2")
assert (
node1.query(
"SELECT engine FROM system.tables where database = 'replicated2' and name = 'example_s3_engine_table'"
)
== "Null\n"
)
assert (
node1.query(
"SELECT count(), sum(id) FROM replicated2.mysql_schema_inference_engine"

View File

@ -6,5 +6,12 @@
<secret_access_key>minio123</secret_access_key>
<format>CSV</format>>
</test_s3>
<test_s3_with_default>
<url>http://minio1:9001/root/data/data{1,2,3}</url>
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
<format>JSONEachRow</format>>
<structure>id UInt32, date Date DEFAULT 18262</structure>
</test_s3_with_default>
</named_collections>
</clickhouse>

View File

@ -459,3 +459,51 @@ def test_cluster_format_detection(started_cluster):
)
assert result == expected_result
def test_cluster_default_expression(started_cluster):
node = started_cluster.instances["s0_0_0"]
node.query(
"insert into function s3('http://minio1:9001/root/data/data1', 'minio', 'minio123', JSONEachRow) select 1 as id settings s3_truncate_on_insert=1"
)
node.query(
"insert into function s3('http://minio1:9001/root/data/data2', 'minio', 'minio123', JSONEachRow) select * from numbers(0) settings s3_truncate_on_insert=1"
)
node.query(
"insert into function s3('http://minio1:9001/root/data/data3', 'minio', 'minio123', JSONEachRow) select 2 as id settings s3_truncate_on_insert=1"
)
expected_result = node.query(
"SELECT * FROM s3('http://minio1:9001/root/data/data{1,2,3}', 'minio', 'minio123', 'JSONEachRow', 'id UInt32, date Date DEFAULT 18262') order by id"
)
result = node.query(
"SELECT * FROM s3Cluster(cluster_simple, 'http://minio1:9001/root/data/data{1,2,3}', 'minio', 'minio123', 'JSONEachRow', 'id UInt32, date Date DEFAULT 18262') order by id"
)
assert result == expected_result
result = node.query(
"SELECT * FROM s3Cluster(cluster_simple, 'http://minio1:9001/root/data/data{1,2,3}', 'minio', 'minio123', 'auto', 'id UInt32, date Date DEFAULT 18262') order by id"
)
assert result == expected_result
result = node.query(
"SELECT * FROM s3Cluster(cluster_simple, 'http://minio1:9001/root/data/data{1,2,3}', 'minio', 'minio123', 'JSONEachRow', 'id UInt32, date Date DEFAULT 18262', 'auto') order by id"
)
assert result == expected_result
result = node.query(
"SELECT * FROM s3Cluster(cluster_simple, 'http://minio1:9001/root/data/data{1,2,3}', 'minio', 'minio123', 'auto', 'id UInt32, date Date DEFAULT 18262', 'auto') order by id"
)
assert result == expected_result
result = node.query(
"SELECT * FROM s3Cluster(cluster_simple, test_s3_with_default) order by id"
)
assert result == expected_result

View File

@ -0,0 +1,33 @@
-----BEGIN CERTIFICATE-----
MIIFojCCA4qgAwIBAgIUBfEMZ1Z/4weV13ryVA9qyNTPJHEwDQYJKoZIhvcNAQEL
BQAwUjELMAkGA1UEBhMCUlUxEzARBgNVBAgMClNvbWUtU3RhdGUxITAfBgNVBAoM
GEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDELMAkGA1UEAwwCY2EwHhcNMjQwOTAy
MTYwODI0WhcNMzQwODMxMTYwODI0WjBXMQswCQYDVQQGEwJSVTETMBEGA1UECAwK
U29tZS1TdGF0ZTEhMB8GA1UECgwYSW50ZXJuZXQgV2lkZ2l0cyBQdHkgTHRkMRAw
DgYDVQQDDAdjbGllbnQ1MIICIjANBgkqhkiG9w0BAQEFAAOCAg8AMIICCgKCAgEA
zivZ5IrYyoJeDX0Zbl/cl8rKE0LbmtD+QKZvQXHD+gstXWxPOEFZVxO3BuqmvBZ1
MaYNyPYA9JyyP+pO9cE8RrTk3w9xMhv8dYWJQK7na9W9RTSXs8xhPwiEm4XuVgqv
GfK/EvdxbFMhgzExOR333TkmXPxrMm5xPWwV3RsTCjNVW7nmdPHXmchuTO7lQtww
6fETqc1Zqv8TO2x/uGZiwAzNYbueWHyzL4Y1UJ7D0mnNNgJvvxtcrzBTlGLLhJ5o
0+zVQLaOnac0WVk0wDhVaxd/gR4bYN3ixvuFbbOaTroFVTMVme196G2FkJI/05Pn
D68r1yUpvuNDjkBbuBO43PlsFKLhPU5twfu4TesEb2WZ0VsNxn8Hc0Ne02WwHsHa
Fi1N0oXvVIeGvvzevuTkjPbh2aCaQX9qbhLXjhgXNFAXQq+qI8ECCWC4LruPL1Es
+ZM2thQAg4k8GY4U9Q8zX55Ut7t9T771QbHFOAupuTgZJSz0jC8JId0m7NOOqtvu
4w/GqefCt9bp+wlQDXVOgi+S3GqKM1OuIbs5XQJtNfi6l684ptAFHSHMHXvfGLHI
MlKdobjNqEFVK9qO/3YvnTUyRyaUA61aHxraXWBwvcUByXwXdgbuz/M5XAi6fr0V
Trw9iUSviDvu4HwDo8PCw9i8xYFtkr2k1kurWEsFZMcCAwEAAaNrMGkwJwYDVR0R
BCAwHoYcc3BpZmZlOi8vYmFyLmNvbS9mb28vYm9vL2ZhcjAdBgNVHQ4EFgQUENIS
25yehLQrlL8vC+DpkNE7urowHwYDVR0jBBgwFoAUiSo9XUmDdI1vjLtMRUK6G2RZ
kKUwDQYJKoZIhvcNAQELBQADggIBAADH/LINEwB1WUQ2Q/aHiNv1ZyJ+ifgs9ewp
/t0uwS+53ctcmJ6Sqeni3/UIdLYjpcdBbl1XpqTYuGF28C2YZrWKFB0HaOiH6D8B
zcGGAkIKFqnhcJxyl37je+scZ8Wk9b04L+X+miN6cgIWm6rQezDF3rs1xvAVBqTM
YPIk6sBIgHNJy4D3S5KdhqNV0/8KY6T65nGFdEq064qOk8HvS6DyYOs22AitCD+L
gcWGJHJ3BfNASbRrT25zb1HLUIFFbFIGaPFd9GbiU5hGb9MgUzX44q+WdXoEa59a
6y9ZcidjEqAGP/FMz16D831YpqRBherZ09ztWXeTfv4NxauisLuoqpOr7CmpQ+Ct
O5t0cUHILeNBFR7rdMOmDawpEcOSGqcJHdPH4SjP/LtgQODWiNys19Yp5afbM5Lz
IjLjq1wAHVtSvPHjRhnZSq0SiU1XlDmu1Em3HbFe5RmqL/lcLe7/U10ddngADG7E
XgPE0jcvl7rYASqYuTbKd6Q53QYx0K7xc1n8mIRJuAofPwl6Yns/ytvw0+E9TBS1
oGb7j6V/k+Xd77dfJ6fckJXPg7Fm3GPO1ax7FNU51sCrvAHsMZhiWQa6pZzBEORM
4yI+DSFyskyWXCPth9r3UqHQXzX86LRkyDWg9l6v3NWRSI1j/e7dZds/U/sg2maq
css4A+kM
-----END CERTIFICATE-----

View File

@ -0,0 +1 @@
subjectAltName=URI:spiffe://bar.com/foo/boo/far

View File

@ -0,0 +1,52 @@
-----BEGIN PRIVATE KEY-----
MIIJQwIBADANBgkqhkiG9w0BAQEFAASCCS0wggkpAgEAAoICAQDOK9nkitjKgl4N
fRluX9yXysoTQtua0P5Apm9BccP6Cy1dbE84QVlXE7cG6qa8FnUxpg3I9gD0nLI/
6k71wTxGtOTfD3EyG/x1hYlArudr1b1FNJezzGE/CISbhe5WCq8Z8r8S93FsUyGD
MTE5HffdOSZc/GsybnE9bBXdGxMKM1VbueZ08deZyG5M7uVC3DDp8ROpzVmq/xM7
bH+4ZmLADM1hu55YfLMvhjVQnsPSac02Am+/G1yvMFOUYsuEnmjT7NVAto6dpzRZ
WTTAOFVrF3+BHhtg3eLG+4Vts5pOugVVMxWZ7X3obYWQkj/Tk+cPryvXJSm+40OO
QFu4E7jc+WwUouE9Tm3B+7hN6wRvZZnRWw3GfwdzQ17TZbAewdoWLU3She9Uh4a+
/N6+5OSM9uHZoJpBf2puEteOGBc0UBdCr6ojwQIJYLguu48vUSz5kza2FACDiTwZ
jhT1DzNfnlS3u31PvvVBscU4C6m5OBklLPSMLwkh3Sbs046q2+7jD8ap58K31un7
CVANdU6CL5LcaoozU64huzldAm01+LqXrzim0AUdIcwde98YscgyUp2huM2oQVUr
2o7/di+dNTJHJpQDrVofGtpdYHC9xQHJfBd2Bu7P8zlcCLp+vRVOvD2JRK+IO+7g
fAOjw8LD2LzFgW2SvaTWS6tYSwVkxwIDAQABAoICAAxYkuU9LUs/MEulfKLf6bFL
SvuJSuZD8p3DebEVE3PDPJW2GS5Yt3Vf8FthLNfhjmYJOBUbKiZ7xa7PJKhRyH7B
cV0sKw6hn2YYN4ZgkuohtAJTamwxZndkVrCLfiC35hFrOGb121xtw/l7tiNh+IhO
Vk5GLVMpSu2vLcX+S48WUNu7z6GcI9mJJ5XCzobI8cLolVFbQy3atXefASz1MJ+o
hJoAJrTMztwfMM0hnL9aZ5f+4Fs008GH5tFhcyd/Zu6d5Y1JAVROgXOCRIboOLE/
iHqeZ2xPDTf2MuDFWw2w1QXrl7UOhDYpbNCh+WF9n14QktMA3jWYnWCE2Rk2USEK
2QhsScNgA7dY3S3DbSK1ZiyZEgbvdq6EYy5I46jNXqRrfKfuD7J04dtO1pmBWZy3
a765FMiejbHrC1xrKtSnvGj1oJ23P3TRiEL6tx3QF4bAulWE62ULBfDgqr56pPbt
KOfPinEFa/14+B4a21iFzsoA6SWPt+i/k00RyRHw4X7acTd8lsavHcT8PGY4wMLX
cNEombZn4h/0oCqAmBtoXYwctcyQwzhJNzAp1GFMG2gdBPhXUsWcwwjNNzeaewCq
BO0YO/DFgj1qTHZwswTfppJMF+BTE/sol4eaqOV1TYV48OfYTdo+6dqfH2JkaJyy
PVJAoZtKKPfDRlrHMrE9AoIBAQD1oZMIi4Nm4TpdOGsU5O2keDW6DXdoxpOI6u9H
YqWZlSZ57q2u+8JkoVJhH/jq4l1QuXtUObpQ9DYOpPwPkt+ZblUut/ZrmMo+gs81
RtIv4EGmgFmtrXgmkYGsoNrIu4/ayBPDwGz7Z+gEta+gxhYbxSZgb2ZOZzH/c2my
3CvfgFiYyv/cfVvynjprRZoxowQvJoMCEEA9REZdO5T51lshQFcrpFU2ucQNKV60
DELV6uJzhL0EDvg9IqP1CxZhdCsTifE/SNVepNWXFegFGVsD/vxmXVxJYevSDQZY
SvGWVcgyuQA8Gdze6y3jOUlzCQ3v7D63PgAPS+yrGXLd0Vz9AoIBAQDW39oRb1n2
ves41cEz7qPNiGmc+zhzyZE30HFnl/RxREPfhzvifTUk3YTUPdJGpcgf+sMoPQ/R
plx/ZrFPTyV7U+svf/eoPlpNnVnZ1m+C/nN7AjVwq04GRHKYD3lTD7KpoYQ0ICGO
z9xFiGbK4VRNIiNpyGc3IZBmrwB2wsGn1L4yfFZly3Th4qiDmNpR29rgteQYWSNJ
hSqsCFcqogJfppP+QKRGf00uHDa1AGQhB2eFykyAniQw52FcdpUdkGqYiR1vdir7
5XckNixq7yyRflo8CftHybyHipTBZVXBzdvDpY5mxANPcJlSPhJxqruxK9FzNYv6
OiDlFnFPwNYTAoIBAQDLByRbWcXhEaWGTA3mlVsAKQRvppXemtRzxYzGOnmaure0
7V3OVvZU6ysGcMD0lOqAwO95rMWeCF1uOVzHGqWLUrFCDni2cvIoPUM8TC+LtxdM
oqZ9cfbwGjFPGR398Vp0ghAkKzdpjncu/WYPw+ueRJT1/N5ZS979wM3LM0VoVdPl
Z1WZUFzh63tqE3viL1ZUCscau4f9nCN1CLyYzV9B2sayogB1XZL5Ngq7U68i1iUb
SspZQm5ZSfqvntx+7OB2I/yuTGtg8WpCma7QGCcs5GSHz/9qAHBFjNKDxF5v3rO9
iUDybIYuE8I4IK/fT8qfV2x3Vd4CxsT2n/Bb/KOZAoIBAQCyIwrqgWIYCQNFLxDp
OOJbtse3+R2oAGd/2jLqKzPf4zRR0A95iUvRsEQ6oLX/E9mBiabZaSYALOdqR3yf
v4HXaI8F5hUvSeIbjmO7EOeJteGPDRm4uowI7h4CqnFuxwjbUKgFfs3TU8fNbXOq
pnv5JmAthpLrRcwtFNBRpMxfkyPLPwFxiubvjbUexE3ap2Yh/SmIdf2nKdtim9eH
5KALJFJ06qpGN6uImqNQE27vYvAUHs6lonVmhaxVt4mP5PY6VxIsIc8o3eeUAcV5
MafFRvcP50aHEVxXEjCY1KXv8fZLkKkp0T7dUQALCqLH0T+hdi5CURYm8KHsylpO
QBQ5AoIBADXNlx5bXEU2lW21wyWG4v3gqvmnDgQG66RXaHC0Rgpn6BW7pose40fv
I82yWviabk7wlvaQgXllzZ5vGL43/8pp+wFpRUl5nX8P1ZA8pRkRArm1vLxQRjkM
90j0M/XhTnC8eC4qKsQx8dPUH0SPkHJAd2tWcP9Q3635zETcINkpsNbsB/QNGBZ1
JefHA/GNZZBrjFMOnvzEbEbqBdhwlZNRlZxVq/svvNzzv12jFQpM7HVy95bATiZ8
x7SQlDxCWOWApd9VpM83JnPWCfu0Yi/HHPJd6YdueMhRSvUpLs1sD8fs9bFQ7+OL
erqwgB4b3yDlXijY75pPDxdPf/0qA3Q=
-----END PRIVATE KEY-----

View File

@ -0,0 +1,27 @@
-----BEGIN CERTIFICATE REQUEST-----
MIIEnDCCAoQCAQAwVzELMAkGA1UEBhMCUlUxEzARBgNVBAgMClNvbWUtU3RhdGUx
ITAfBgNVBAoMGEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDEQMA4GA1UEAwwHY2xp
ZW50NTCCAiIwDQYJKoZIhvcNAQEBBQADggIPADCCAgoCggIBAM4r2eSK2MqCXg19
GW5f3JfKyhNC25rQ/kCmb0Fxw/oLLV1sTzhBWVcTtwbqprwWdTGmDcj2APScsj/q
TvXBPEa05N8PcTIb/HWFiUCu52vVvUU0l7PMYT8IhJuF7lYKrxnyvxL3cWxTIYMx
MTkd9905Jlz8azJucT1sFd0bEwozVVu55nTx15nIbkzu5ULcMOnxE6nNWar/Ezts
f7hmYsAMzWG7nlh8sy+GNVCew9JpzTYCb78bXK8wU5Riy4SeaNPs1UC2jp2nNFlZ
NMA4VWsXf4EeG2Dd4sb7hW2zmk66BVUzFZntfehthZCSP9OT5w+vK9clKb7jQ45A
W7gTuNz5bBSi4T1ObcH7uE3rBG9lmdFbDcZ/B3NDXtNlsB7B2hYtTdKF71SHhr78
3r7k5Iz24dmgmkF/am4S144YFzRQF0KvqiPBAglguC67jy9RLPmTNrYUAIOJPBmO
FPUPM1+eVLe7fU++9UGxxTgLqbk4GSUs9IwvCSHdJuzTjqrb7uMPxqnnwrfW6fsJ
UA11ToIvktxqijNTriG7OV0CbTX4upevOKbQBR0hzB173xixyDJSnaG4zahBVSva
jv92L501MkcmlAOtWh8a2l1gcL3FAcl8F3YG7s/zOVwIun69FU68PYlEr4g77uB8
A6PDwsPYvMWBbZK9pNZLq1hLBWTHAgMBAAGgADANBgkqhkiG9w0BAQsFAAOCAgEA
SGJaGaSkfsUBSqPbkbtyE2ndHIY8QMPvJAMB7pPbA/MlX39oO2U1AraFatN/Xzpd
0uQA2m0UsXF2+ScrG4rd14YkU4wpcT2lIplvFFzUkwfYLATRYrFV0FCru1n28+Ve
TyUG9/tlf/jnNZwWtB5AEbCWmzXSjXUuwXdn2oQ0z+3dxdSnpOrDkzyYeocIMnMJ
pG0qdBd+hJrK6snJNli6EfsmmykyCSLorFpnkm2uKUwdTAD2/MlxYdutjk7tqKw5
wpYi6Wqt/euyQ94Ri4aICZMBFk5+zDdaEOKVQZ0aUT8RWt/aD2ksbTrclDICOe84
iG+Nf/CyNqSNqsUugCSCxN3UUZgLKEyJb8Pz4N/nFFRBb1aiZ/5YVRTYtZ+k4tLb
s2exxqVpDg3M5G5bk0iJ8DBEjO/yKwobc7HAtnTAEIs0HGord3yzg3F0Y+5ecQAg
9ESOptz33EBkTHxpBmt0D0ACa4CTghrf/Id8imNtdGU9i7MeEba+iUVAUP8VfhtL
FJPRR8aVaKaLc9uCAiiHuRc+J0EHAwTOKKTK3Z1mkYO00kMCiRcru8/H6ibkrkV7
kRL6NvAc7CsEzSDPkFKAZhQ4p6AcfNC5yRiNLG9JB/wQgBg8v23Uwtk74gOXIaQN
WUjwzdYOljdcFOzmoDMzyfIFwSXcO3dmmjqOwh2HNw0=
-----END CERTIFICATE REQUEST-----

View File

@ -0,0 +1,33 @@
-----BEGIN CERTIFICATE-----
MIIFojCCA4qgAwIBAgIUBfEMZ1Z/4weV13ryVA9qyNTPJHIwDQYJKoZIhvcNAQEL
BQAwUjELMAkGA1UEBhMCUlUxEzARBgNVBAgMClNvbWUtU3RhdGUxITAfBgNVBAoM
GEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDELMAkGA1UEAwwCY2EwHhcNMjQwOTAy
MTYwODM0WhcNMzQwODMxMTYwODM0WjBXMQswCQYDVQQGEwJSVTETMBEGA1UECAwK
U29tZS1TdGF0ZTEhMB8GA1UECgwYSW50ZXJuZXQgV2lkZ2l0cyBQdHkgTHRkMRAw
DgYDVQQDDAdjbGllbnQ2MIICIjANBgkqhkiG9w0BAQEFAAOCAg8AMIICCgKCAgEA
uz+3VyFhBD6K68mkDM0LJfRfGbuD8tSmHhtO6+XQzhIUiANW+A1WdyD+zWky0QsS
vl/2Ds1PmjU7659pkSD8Sidjdz3/TP0eO49nHinLjRQa2Oawk6PCjLIcpJ+A5eGb
Hno/oiQBMCAPpVh3sex+5yPiUQN62cKRWryv2JJqJmEgvpjC92SaIjf9M3mCsxMf
an0CZj6hNcorxXQNnF4JZRPQ4mMgBqgFS5Oz1YujHiBUN9ZoIGmS3HZ9LYl5lL7o
GxNXluyIrIw1kiyQrx+wJFdYwTBeyCqw8wmf993tHRmvpJ2ZFVXTbYqpj2Qkom+J
EpCRBqCmK+/uSgLOAicS/wR2eiYXkw2rYDIQ6yCyeW7nnaFVWNAFNSdAW1Jdr+Z2
fKMolhYAcpeMQAYN5wTqJBF9dDwMxbOcdSh1wPZcenuO2q5cMJcn3qt3+SigNKcQ
BvqtZ54GNGq9h24f11+cqco80v4WYxrmXu8bXQfgtX07UVvqkjbG7O4HtbGDvOlw
KO7d1kOj4XUJdZbz9g5kaGufN4hlBs9JzYNOZdywNwBiPxHHE71Ht2ihfbBKSl9x
4Zse5YhPqToacWd5FRW+If5EpHkxxW+f4e61S2/8fnn5yHpMX22HXNBmOUR3vBrl
xfkje22ZuEf5NfB95aEaRZABmXQaHKdAVQeaAT9TvPMCAwEAAaNrMGkwJwYDVR0R
BCAwHoYcc3BpZmZlOi8vYmFyLmNvbS9mb28vYmF6L2ZhcjAdBgNVHQ4EFgQU0ieN
0CTYiMz7HJs9OH2U4imSzN0wHwYDVR0jBBgwFoAUiSo9XUmDdI1vjLtMRUK6G2RZ
kKUwDQYJKoZIhvcNAQELBQADggIBAELD77yWem26DIWG2Mi6q9KvLb/d8bOFN4mg
SYYekdEryyYCFhbb0P9T7GKj4KxNxkZaGgwXl+PM8SH7FhqemRCwCtHMVQiWvC1x
XLCrGiep7Dx+vvKHoCdapQp3uRfsZMHSZbsHJgQRFRt/jgP3jFcrLAa9tD8Yza4y
lBhh8pUROC0edlqom+BkLYA6A7FWA89x/NZL6wiyPKqRh2SuACszrlQdj4oqdgIF
pAILjDy4fLaCGGUgu9kHonV0xTjqSdFXiSeImSkePOvGWrh2B5GqRbanPHDSR5VD
pIhKZPpnvUBNhzoAMv1RFbgNs9m3sYqYChxb8IOGiY/3EG4rgygVez2yu16v5WMU
PIWwSv3N+oceG085dHxhn4TcDamdFrQHssp7GJBbyGEINHQRiZ4cu3Vz2cVNYcKD
iFJZ8vVwU7IZOEQeww6DT+gL+wqSgTyDvEXQNbYupFYPZXSBIaygH4eHa+PqPnNL
DJTpPAlwNRB2+eL3bZxvNAfwqIY6xgwnLBr1QrosmfMKgkswkg8gVoIpIRGACzk2
iY818Jn+IG/M/aPF3p5dTsOqH3bQmz4ZpoLB2dytqkNDGKSAPPGOpe4MDyx0prCH
GWDjEjn4xT9AjAJVpTWJCENPzFzwA7byApuZwLTy/5ZBbNJf1K/JwsqXUhHKENb2
NzMKvQCT
-----END CERTIFICATE-----

View File

@ -0,0 +1 @@
subjectAltName=URI:spiffe://bar.com/foo/baz/far

View File

@ -0,0 +1,52 @@
-----BEGIN PRIVATE KEY-----
MIIJQwIBADANBgkqhkiG9w0BAQEFAASCCS0wggkpAgEAAoICAQC7P7dXIWEEPorr
yaQMzQsl9F8Zu4Py1KYeG07r5dDOEhSIA1b4DVZ3IP7NaTLRCxK+X/YOzU+aNTvr
n2mRIPxKJ2N3Pf9M/R47j2ceKcuNFBrY5rCTo8KMshykn4Dl4Zseej+iJAEwIA+l
WHex7H7nI+JRA3rZwpFavK/YkmomYSC+mML3ZJoiN/0zeYKzEx9qfQJmPqE1yivF
dA2cXgllE9DiYyAGqAVLk7PVi6MeIFQ31mggaZLcdn0tiXmUvugbE1eW7IisjDWS
LJCvH7AkV1jBMF7IKrDzCZ/33e0dGa+knZkVVdNtiqmPZCSib4kSkJEGoKYr7+5K
As4CJxL/BHZ6JheTDatgMhDrILJ5buedoVVY0AU1J0BbUl2v5nZ8oyiWFgByl4xA
Bg3nBOokEX10PAzFs5x1KHXA9lx6e47arlwwlyfeq3f5KKA0pxAG+q1nngY0ar2H
bh/XX5ypyjzS/hZjGuZe7xtdB+C1fTtRW+qSNsbs7ge1sYO86XAo7t3WQ6PhdQl1
lvP2DmRoa583iGUGz0nNg05l3LA3AGI/EccTvUe3aKF9sEpKX3Hhmx7liE+pOhpx
Z3kVFb4h/kSkeTHFb5/h7rVLb/x+efnIekxfbYdc0GY5RHe8GuXF+SN7bZm4R/k1
8H3loRpFkAGZdBocp0BVB5oBP1O88wIDAQABAoICACymDcaV6/dCHogIpbhzsAPV
2FNfdiAR+qZVJqVzQC3S+9hOy48MRyDS2k8KlZZpCIgig56V8DQ6G1acxWRYtC/O
YpZNTzIBbRMQp6r2llXGhHxRzar2sm4wDkpmyiqGeCF1TvUPlsTt8C0iAjCHzt64
nL9qkAGatmQnd9qxVuRd5pvr+xlYgLRGG3cJs1OV7LjMpCTTLEeSNIu5l4FAnwbe
CcHhlwJfUBvsVUZHLJcDaHGEC4InCmDNVB3mmPoR53CFVS5kqlZSfHelbr6DVNHl
jgFK0l7yZw0cr2tAHMkClfIvrg/7ThXhqXrKiz28ULf/hsVIyzbQ2EYHky3KOWny
04O7/NnOkVHs+XUyNC4sv9nkcd9ntKkvvUPPK0U6vbW7IasC3jCh5LMyZjHYwgmK
hzxEBZSyutKWn3RWncarwQ/1Vbq3HjbkeVTipXAa7Bny17wiAeZMZ2GqQZ9VcNQ3
YJWDgxS5shwcEo+71sC4o2HjmWKcPujmt84XcWc6yphPbCpbwwswaQD5MaZxeDgZ
OUhF9LfslzNrwwoZTFz/Qhy3TOshF7BIbUdQnWLMNdAb9ccby/0WgOmfD6V4t99N
ksb2nWgvvK3isycs6HHVP/fgv+yM9cKGs66JoH2Jm+SInUtpR5Gv1aGeV97/9WFd
JuiHtHQIty+8n6GDTscJAoIBAQDuHCBeZ+pVTyW6wxjd4OD2keuDjM3Z7X/UgCoJ
kR87Dwjd8SHEw8QaH8vvGYBNktFu3KbQ1TV2OR8gAdnwlHeI5V2/nIVX0UBjQM9X
GC3cmzsMOBAem0nuYXZG9yvawwPUdZ18fQc2wAs4GqL4uKaOeuCefNyK5wKfvX7M
sA49D45gvLUhpfkTeM8HK9UQwMfMg2fFBzZifqTIG4OGkkAeEY+rkJTUxnvTuuFU
dkXXF8Qe+pSPkbQVQYYRRO9Wk0i16R6VaYrl3vvi72w2gEw7iQya0A1bHZe3s7vv
jQuz8h954kcgLYCqsOm/mj3t654jrjW1Z5yRjznTUJKrKMh3AoIBAQDJUVCp2Frm
NgzrZXD1QrkJ1qCRBHyVu7FikXqNszc9lLD5y8YWRnhDtGruOQ3DYjpuD/DMrO2P
+iBTambM3mJt6FE8TkXHyMzLoJ/I8SMLMbLNdDpsj8D8RlftwIESiNu9DQfMle5l
8jxZ7R7usio8HysVm5u6fsSmYVUZF+sWLLAUqote4HQxdvDup9A1q7onVZUYfKnK
mCVKqfdqFDqMKTSHOngxA5wzQWdMqdgLxiYKPYbkNsJ3dhXQwJjfbyDQq4X/foec
0wG91/WqsLiMOZLsQBiGMgOq85IqGBByl51QnkT8drPEJsXX6UCHjQ7AYHe0U+pe
JTa6nMfk2AplAoIBAQDemJa+CuFrvSEldowBqOBGQeXtRi2PBNNTAjnKVcvfd0+v
xGPwQZ9LWNxaevmSC6YUwNyGMDvZeM8bKf/nQ7R32w0IOzgA/9L0trrezfDZ4piR
9LtFEaFM4/ohn6J00Yj8DrQak/uxeFlEqsoeQOkcG81u/IVkqU+vrAQlMJUvCiLt
VpzyhunSBUdtidGW5tIh49qXvAVgkMpVdDtCC+k68unf1rr8K03Jg1RxlFv4F/S1
jUZi7TBwCqBd9pbU1b3MqdF4loFOImZSIceFL+2UXqvU8pj5zDFwf+s6qB3/rGM2
m44oi8FUUS1EfNpWWMWuz4bQPruE7GN/pDxpHChDAoIBAGZF5yLCBTvazalwuXf/
051J6fyCOQCgf7dBjO8b0r54IYhlm1aJqmNK7t/jIDjYWDK96mkwmOeB/JYkAeGm
QH7xfQOUCCM8wb3Y9hPRXSo8r0ds+plYVoRTACyKlYfi+y8JxaKLsLcd3scYjZRZ
8tbkRrENgom2CRU1cVP0MLvtK+7zzSYABUdz02aK3l3FxiZhdgMgoemIbbmGq2i6
qhu2ezcP3yuXV+06Fs59ooIowf6Fz1d08kpaNS7+CSvJevRHjyWDFEX5dHMLmYSD
jt2+CgP3c/4IvpBEeUblPsXYfFUOcqGHdD/8KppLpzq10H6vA2EbGH+HjEw1hd+e
WXkCggEBAIq+MHXpbipKp7lsxsp/moeNXUojLfiBcQLU+zauWBdP3kTJmDd7D3Vy
MoN4zpvhboue70RP1ZZG3yp8s9yR62XrO+44DrKKIhVitpa1IPSavFovFVl/vx1H
F6PuZvACBF7mRZa23h9XSizvdfycEDf3rZidLG/Y9IarrLF3HYjClBktJTtm0YaU
QnmTSUdNvRBCs6/BGQNXY76VB5AHNjeLjjuLLbxYF5XsHlsdacgdMv2ShmIcibrT
nSMK3RdRdRt1Nu1k6v7MkGcaSky7zJIeu6+VftA/7bVKBcnvb+iGZSUSk2QRTCGW
nT+c65hmPp61jcBOgCF3CntcIw4eEDc=
-----END PRIVATE KEY-----

View File

@ -0,0 +1,27 @@
-----BEGIN CERTIFICATE REQUEST-----
MIIEnDCCAoQCAQAwVzELMAkGA1UEBhMCUlUxEzARBgNVBAgMClNvbWUtU3RhdGUx
ITAfBgNVBAoMGEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDEQMA4GA1UEAwwHY2xp
ZW50NjCCAiIwDQYJKoZIhvcNAQEBBQADggIPADCCAgoCggIBALs/t1chYQQ+iuvJ
pAzNCyX0Xxm7g/LUph4bTuvl0M4SFIgDVvgNVncg/s1pMtELEr5f9g7NT5o1O+uf
aZEg/EonY3c9/0z9HjuPZx4py40UGtjmsJOjwoyyHKSfgOXhmx56P6IkATAgD6VY
d7Hsfucj4lEDetnCkVq8r9iSaiZhIL6YwvdkmiI3/TN5grMTH2p9AmY+oTXKK8V0
DZxeCWUT0OJjIAaoBUuTs9WLox4gVDfWaCBpktx2fS2JeZS+6BsTV5bsiKyMNZIs
kK8fsCRXWMEwXsgqsPMJn/fd7R0Zr6SdmRVV022KqY9kJKJviRKQkQagpivv7koC
zgInEv8EdnomF5MNq2AyEOsgsnlu552hVVjQBTUnQFtSXa/mdnyjKJYWAHKXjEAG
DecE6iQRfXQ8DMWznHUodcD2XHp7jtquXDCXJ96rd/kooDSnEAb6rWeeBjRqvYdu
H9dfnKnKPNL+FmMa5l7vG10H4LV9O1Fb6pI2xuzuB7Wxg7zpcCju3dZDo+F1CXWW
8/YOZGhrnzeIZQbPSc2DTmXcsDcAYj8RxxO9R7dooX2wSkpfceGbHuWIT6k6GnFn
eRUVviH+RKR5McVvn+HutUtv/H55+ch6TF9th1zQZjlEd7wa5cX5I3ttmbhH+TXw
feWhGkWQAZl0GhynQFUHmgE/U7zzAgMBAAGgADANBgkqhkiG9w0BAQsFAAOCAgEA
Gewd4gSxvJJ1LIKLVTLeMdEdJfzsS52Kh3BCerku/C4ZKcIyT49dTbi6l6d40bHJ
Cs32Hwps8/qufhwwWn0r/wyww1Mgfr6ccMgKmrz1VbgWmD9owDzlL014ygiDk8yi
LwfSLe43NFrFC/FcAJUd/P8UEe0/8GcHjzhU/zqh3VAL7RzSL3k73CsTFiDaxtQL
8qISpA0lYTldMx5RaN9COWi0rPFv7mJAYRXtE/Cb+T2hb53khOiiRrJYIEJjjAhj
g8p9FOzlbXdpfoChVk7NA90CbCbxrQ2BiUqQAVYnGhntzHMwR9YxOYjvjnuiHuHX
7+4Vheda88TciMJlj0TC2e1mXBo182n/qfETeI26MsEOs3DidLT+ygM3woFQyIrX
3x2kDlvmILKg1mPGhqaRwwzCmk5a1TVdDTRo9VkOvR5/tbfG3vHlgpvBtUFCkEjX
HOyRh0A3TquofUbtx638lMWscBLqM5g6VO+Hytk6zBmq+8caJFNTOeTHZur04ZLM
SWfkIwl0B863owNuq4KxXI3NvpCc5LtGc9UrwVoHSH/pv6tbKEX15Y0ERL5/e33M
GT0D00cPnWAzmYQpYzHQ3Dj29XKlIxWBkn1QvrIFyb/T+dld1efZ3HlQxZEQvOsR
McY90r+HmVt8uCioYnC4DmchWlSX1MJe/h72udVbAXk=
-----END CERTIFICATE REQUEST-----

View File

@ -14,12 +14,16 @@ openssl req -newkey rsa:4096 -nodes -batch -keyout client1-key.pem -out client1-
openssl req -newkey rsa:4096 -nodes -batch -keyout client2-key.pem -out client2-req.pem -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=client2"
openssl req -newkey rsa:4096 -nodes -batch -keyout client3-key.pem -out client3-req.pem -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=client3"
openssl req -newkey rsa:4096 -nodes -batch -keyout client4-key.pem -out client4-req.pem -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=client4"
openssl req -newkey rsa:4096 -nodes -batch -keyout client5-key.pem -out client5-req.pem -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=client5"
openssl req -newkey rsa:4096 -nodes -batch -keyout client6-key.pem -out client6-req.pem -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=client6"
# 5. Use CA's private key to sign client's CSR and get back the signed certificate
openssl x509 -req -days 3650 -in client1-req.pem -CA ca-cert.pem -CAkey ca-key.pem -CAcreateserial -out client1-cert.pem
openssl x509 -req -days 3650 -in client2-req.pem -CA ca-cert.pem -CAkey ca-key.pem -CAcreateserial -out client2-cert.pem
openssl x509 -req -days 3650 -in client3-req.pem -CA ca-cert.pem -CAkey ca-key.pem -CAcreateserial -out client3-cert.pem
openssl x509 -req -days 3650 -in client4-req.pem -CA ca-cert.pem -CAkey ca-key.pem -CAcreateserial -extfile client4-ext.cnf -out client4-cert.pem
openssl x509 -req -days 3650 -in client5-req.pem -CA ca-cert.pem -CAkey ca-key.pem -CAcreateserial -extfile client5-ext.cnf -out client5-cert.pem
openssl x509 -req -days 3650 -in client6-req.pem -CA ca-cert.pem -CAkey ca-key.pem -CAcreateserial -extfile client6-ext.cnf -out client6-cert.pem
# 6. Generate one more self-signed certificate and private key for using as wrong certificate (because it's not signed by CA)
openssl req -newkey rsa:4096 -x509 -days 3650 -nodes -batch -keyout wrong-key.pem -out wrong-cert.pem -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=client"

View File

@ -17,6 +17,11 @@
<subject_alt_name>URI:spiffe://foo.com/baz</subject_alt_name>
</ssl_certificates>
</jerome>
<stewie>
<ssl_certificates>
<subject_alt_name>URI:spiffe://bar.com/foo/*/far</subject_alt_name>
</ssl_certificates>
</stewie>
<peter>
<no_password/>
</peter>

View File

@ -334,6 +334,8 @@ def test_create_user():
'lucy\tssl_certificate\t{"common_names":["client2","client3"]}\n'
)
instance.query("DROP USER emma")
def test_x509_san_support():
assert (
@ -369,3 +371,42 @@ def test_x509_san_support():
instance.query("SHOW CREATE USER jemma")
== "CREATE USER jemma IDENTIFIED WITH ssl_certificate SAN \\'URI:spiffe://foo.com/bar\\', \\'URI:spiffe://foo.com/baz\\'\n"
)
instance.query("DROP USER jemma")
def test_x509_san_wildcard_support():
assert (
execute_query_native(
instance, "SELECT currentUser()", user="stewie", cert_name="client5"
)
== "stewie\n"
)
assert (
instance.query(
"SELECT name, auth_type, auth_params FROM system.users WHERE name='stewie'"
)
== 'stewie\tssl_certificate\t{"subject_alt_names":["URI:spiffe:\\\\/\\\\/bar.com\\\\/foo\\\\/*\\\\/far"]}\n'
)
assert (
instance.query("SHOW CREATE USER stewie")
== "CREATE USER stewie IDENTIFIED WITH ssl_certificate SAN \\'URI:spiffe://bar.com/foo/*/far\\'\n"
)
instance.query(
"CREATE USER brian IDENTIFIED WITH ssl_certificate SAN 'URI:spiffe://bar.com/foo/*/far'"
)
assert (
execute_query_https("SELECT currentUser()", user="brian", cert_name="client6")
== "brian\n"
)
assert (
instance.query("SHOW CREATE USER brian")
== "CREATE USER brian IDENTIFIED WITH ssl_certificate SAN \\'URI:spiffe://bar.com/foo/*/far\\'\n"
)
instance.query("DROP USER brian")

View File

@ -662,10 +662,7 @@ def test_multiple_tables_meta_mismatch(started_cluster):
},
)
except QueryRuntimeException as e:
assert (
"Table columns structure in ZooKeeper is different from local table structure"
in str(e)
)
assert "Existing table metadata in ZooKeeper differs in columns" in str(e)
failed = True
assert failed is True

View File

@ -190,3 +190,195 @@
20 21 22
23 24 25
26 27 28
0 0 0
0 0 0
0 0 0
1 2 3
4 5 6
7 8 9
10 11 12
13 14 15
16 17 18
20 21 22
23 24 25
26 27 28
0 0 0
0 0 0
0 0 0
1 2 3
4 5 6
7 8 9
10 11 12
13 14 15
16 17 18
20 21 22
23 24 25
26 27 28
0 0 0
0 0 0
0 0 0
1 2 3
4 5 6
7 8 9
10 11 12
13 14 15
16 17 18
20 21 22
23 24 25
26 27 28
0 0 0
0 0 0
0 0 0
1 2 3
4 5 6
7 8 9
10 11 12
13 14 15
16 17 18
20 21 22
23 24 25
26 27 28
0 0 0
0 0 0
0 0 0
1 2 3
4 5 6
7 8 9
10 11 12
13 14 15
16 17 18
20 21 22
23 24 25
26 27 28
0 0 0
0 0 0
0 0 0
1 2 3
4 5 6
7 8 9
10 11 12
13 14 15
16 17 18
20 21 22
23 24 25
26 27 28
0 0 0
0 0 0
0 0 0
1 2 3
4 5 6
7 8 9
10 11 12
13 14 15
16 17 18
20 21 22
23 24 25
26 27 28
0 0 0
0 0 0
0 0 0
1 2 3
4 5 6
7 8 9
10 11 12
13 14 15
16 17 18
20 21 22
23 24 25
26 27 28
0 0 0
0 0 0
0 0 0
1 2 3
4 5 6
7 8 9
10 11 12
13 14 15
16 17 18
20 21 22
23 24 25
26 27 28
0 0 0
0 0 0
0 0 0
1 2 3
4 5 6
7 8 9
10 11 12
13 14 15
16 17 18
20 21 22
23 24 25
26 27 28
0 0 0
0 0 0
0 0 0
1 2 3
4 5 6
7 8 9
10 11 12
13 14 15
16 17 18
20 21 22
23 24 25
26 27 28
0 0 0
0 0 0
0 0 0
1 2 3
4 5 6
7 8 9
10 11 12
13 14 15
16 17 18
20 21 22
23 24 25
26 27 28
0 0 0
0 0 0
0 0 0
1 2 3
4 5 6
7 8 9
10 11 12
13 14 15
16 17 18
20 21 22
23 24 25
26 27 28
0 0 0
0 0 0
0 0 0
1 2 3
4 5 6
7 8 9
10 11 12
13 14 15
16 17 18
20 21 22
23 24 25
26 27 28
0 0 0
0 0 0
0 0 0
1 2 3
4 5 6
7 8 9
10 11 12
13 14 15
16 17 18
20 21 22
23 24 25
26 27 28
0 0 0
0 0 0
0 0 0
1 2 3
4 5 6
7 8 9
10 11 12
13 14 15
16 17 18
20 21 22
23 24 25
26 27 28

View File

@ -2,21 +2,37 @@
-- Tag no-fasttest: Depends on AWS
select * from s3('http://localhost:11111/test/{a,b,c}.tsv') ORDER BY c1, c2, c3;
select * from s3('http://localhost:11111/test/{a,b,c}.tsv', NOSIGN) ORDER BY c1, c2, c3;
select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'TSV') ORDER BY c1, c2, c3;
select * from s3('http://localhost:11111/test/{a,b,c}.tsv', NOSIGN, 'TSV') ORDER BY c1, c2, c3;
select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64') ORDER BY c1, c2, c3;
select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest') ORDER BY c1, c2, c3;
select * from s3('http://localhost:11111/test/{a,b,c}.tsv', NOSIGN, 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64') ORDER BY c1, c2, c3;
select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto') ORDER BY c1, c2, c3;
select * from s3('http://localhost:11111/test/{a,b,c}.tsv', NOSIGN, 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto') ORDER BY c1, c2, c3;
select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest') ORDER BY c1, c2, c3;
select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', '') ORDER BY c1, c2, c3;
select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV') ORDER BY c1, c2, c3;
select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', '', 'TSV') ORDER BY c1, c2, c3;
select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64') ORDER BY c1, c2, c3;
select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', '', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64') ORDER BY c1, c2, c3;
select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto') ORDER BY c1, c2, c3;
select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', '', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto') ORDER BY c1, c2, c3;
select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv') ORDER BY c1, c2, c3;
select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', NOSIGN) ORDER BY c1, c2, c3;
select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV') ORDER BY c1, c2, c3;
select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', NOSIGN, 'TSV') ORDER BY c1, c2, c3;
select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64') ORDER BY c1, c2, c3;
select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest') ORDER BY c1, c2, c3;
select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', NOSIGN, 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64') ORDER BY c1, c2, c3;
select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto') ORDER BY c1, c2, c3;
select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', NOSIGN, 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto') ORDER BY c1, c2, c3;
select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest') ORDER BY c1, c2, c3;
select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', '') ORDER BY c1, c2, c3;
select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV') ORDER BY c1, c2, c3;
select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', '', 'TSV') ORDER BY c1, c2, c3;
select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64') ORDER BY c1, c2, c3;
select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', '', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64') ORDER BY c1, c2, c3;
select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto') ORDER BY c1, c2, c3;
select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', '', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto') ORDER BY c1, c2, c3;

View File

@ -22,3 +22,24 @@ Row 1:
──────
x: 1
2
-- Bug 67476: Queries with overflow mode != throw must not be cached by the query cache
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0

View File

@ -36,4 +36,32 @@ SELECT count(*) FROM system.query_cache;
DROP TABLE tab;
SELECT '-- Bug 67476: Queries with overflow mode != throw must not be cached by the query cache';
DROP TABLE IF EXISTS tab;
CREATE TABLE tab(c UInt64) ENGINE = Memory;
SYSTEM DROP QUERY CACHE;
SELECT sum(c) FROM tab SETTINGS read_overflow_mode = 'break', use_query_cache = 1;
SELECT count(*) from system.query_cache;
SELECT sum(c) FROM tab SETTINGS read_overflow_mode_leaf = 'break', use_query_cache = 1;
SELECT count(*) from system.query_cache;
SELECT sum(c) FROM tab SETTINGS group_by_overflow_mode = 'break', use_query_cache = 1;
SELECT count(*) from system.query_cache;
SELECT sum(c) FROM tab SETTINGS sort_overflow_mode = 'break', use_query_cache = 1;
SELECT count(*) from system.query_cache;
SELECT sum(c) FROM tab SETTINGS result_overflow_mode = 'break', use_query_cache = 1;
SELECT count(*) from system.query_cache;
SELECT sum(c) FROM tab SETTINGS timeout_overflow_mode = 'break', use_query_cache = 1;
SELECT count(*) from system.query_cache;
SELECT sum(c) FROM tab SETTINGS set_overflow_mode = 'break', use_query_cache = 1;
SELECT count(*) from system.query_cache;
SELECT sum(c) FROM tab SETTINGS join_overflow_mode = 'break', use_query_cache = 1;
SELECT count(*) from system.query_cache;
SELECT sum(c) FROM tab SETTINGS transfer_overflow_mode = 'break', use_query_cache = 1;
SELECT count(*) from system.query_cache;
SELECT sum(c) FROM tab SETTINGS distinct_overflow_mode = 'break', use_query_cache = 1;
SELECT count(*) from system.query_cache;
SYSTEM DROP QUERY CACHE;