mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge remote-tracking branch 'origin/master' into LessReadInOrder
This commit is contained in:
commit
d05a79536f
@ -109,7 +109,7 @@ if [ -n "$CLICKHOUSE_USER" ] && [ "$CLICKHOUSE_USER" != "default" ] || [ -n "$CL
|
||||
<networks>
|
||||
<ip>::/0</ip>
|
||||
</networks>
|
||||
<password>${CLICKHOUSE_PASSWORD}</password>
|
||||
<password><![CDATA[${CLICKHOUSE_PASSWORD//]]>/]]]]><![CDATA[>}]]></password>
|
||||
<quota>default</quota>
|
||||
<access_management>${CLICKHOUSE_ACCESS_MANAGEMENT}</access_management>
|
||||
</${CLICKHOUSE_USER}>
|
||||
|
@ -1396,6 +1396,7 @@ SELECT * FROM json_each_row_nested
|
||||
- [input_format_json_ignore_unknown_keys_in_named_tuple](/docs/en/operations/settings/settings-formats.md/#input_format_json_ignore_unknown_keys_in_named_tuple) - ignore unknown keys in json object for named tuples. Default value - `false`.
|
||||
- [input_format_json_compact_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_json_compact_allow_variable_number_of_columns) - allow variable number of columns in JSONCompact/JSONCompactEachRow format, ignore extra columns and use default values on missing columns. Default value - `false`.
|
||||
- [input_format_json_throw_on_bad_escape_sequence](/docs/en/operations/settings/settings-formats.md/#input_format_json_throw_on_bad_escape_sequence) - throw an exception if JSON string contains bad escape sequence. If disabled, bad escape sequences will remain as is in the data. Default value - `true`.
|
||||
- [input_format_json_empty_as_default](/docs/en/operations/settings/settings-formats.md/#input_format_json_empty_as_default) - treat empty fields in JSON input as default values. Default value - `false`. For complex default expressions [input_format_defaults_for_omitted_fields](/docs/en/operations/settings/settings-formats.md/#input_format_defaults_for_omitted_fields) must be enabled too.
|
||||
- [output_format_json_quote_64bit_integers](/docs/en/operations/settings/settings-formats.md/#output_format_json_quote_64bit_integers) - controls quoting of 64-bit integers in JSON output format. Default value - `true`.
|
||||
- [output_format_json_quote_64bit_floats](/docs/en/operations/settings/settings-formats.md/#output_format_json_quote_64bit_floats) - controls quoting of 64-bit floats in JSON output format. Default value - `false`.
|
||||
- [output_format_json_quote_denormals](/docs/en/operations/settings/settings-formats.md/#output_format_json_quote_denormals) - enables '+nan', '-nan', '+inf', '-inf' outputs in JSON output format. Default value - `false`.
|
||||
|
@ -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>
|
||||
```
|
||||
|
@ -752,6 +752,17 @@ Possible values:
|
||||
|
||||
Default value: 0.
|
||||
|
||||
### input_format_json_empty_as_default {#input_format_json_empty_as_default}
|
||||
|
||||
When enabled, replace empty input fields in JSON with default values. For complex default expressions `input_format_defaults_for_omitted_fields` must be enabled too.
|
||||
|
||||
Possible values:
|
||||
|
||||
+ 0 — Disable.
|
||||
+ 1 — Enable.
|
||||
|
||||
Default value: 0.
|
||||
|
||||
## TSV format settings {#tsv-format-settings}
|
||||
|
||||
### input_format_tsv_empty_as_default {#input_format_tsv_empty_as_default}
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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); }
|
||||
|
@ -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,
|
||||
|
@ -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,
|
||||
|
@ -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,
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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>();
|
||||
|
@ -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.
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
|
@ -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)
|
||||
|
@ -40,6 +40,7 @@ enum class OpNum : int32_t
|
||||
FilteredList = 500,
|
||||
CheckNotExists = 501,
|
||||
CreateIfNotExists = 502,
|
||||
RemoveRecursive = 503,
|
||||
|
||||
SessionID = 997, /// Special internal request
|
||||
};
|
||||
|
@ -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,
|
||||
|
@ -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,
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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";
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -12,6 +12,7 @@ enum class KeeperFeatureFlag : size_t
|
||||
MULTI_READ,
|
||||
CHECK_NOT_EXISTS,
|
||||
CREATE_IF_NOT_EXISTS,
|
||||
REMOVE_RECURSIVE,
|
||||
};
|
||||
|
||||
class KeeperFeatureFlags
|
||||
|
@ -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());
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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, ""}}));
|
||||
|
@ -1144,6 +1144,7 @@ class IColumn;
|
||||
M(Bool, input_format_try_infer_variants, false, "Try to infer the Variant type in text formats when there is more than one possible type for column/array elements", 0) \
|
||||
M(Bool, type_json_skip_duplicated_paths, false, "When enabled, during parsing JSON object into JSON type duplicated paths will be ignored and only the first one will be inserted instead of an exception", 0) \
|
||||
M(UInt64, input_format_json_max_depth, 1000, "Maximum depth of a field in JSON. This is not a strict limit, it does not have to be applied precisely.", 0) \
|
||||
M(Bool, input_format_json_empty_as_default, false, "Treat empty fields in JSON input as default values.", 0) \
|
||||
M(Bool, input_format_try_infer_integers, true, "Try to infer integers instead of floats while schema inference in text formats", 0) \
|
||||
M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \
|
||||
M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \
|
||||
|
@ -71,6 +71,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
|
||||
},
|
||||
{"24.9",
|
||||
{
|
||||
{"input_format_json_empty_as_default", false, false, "Added new setting to allow to treat empty fields in JSON input as default values."},
|
||||
{"input_format_try_infer_variants", false, false, "Try to infer Variant type in text formats when there is more than one possible type for column/array elements"},
|
||||
{"join_output_by_rowlist_perkey_rows_threshold", 0, 5, "The lower limit of per-key average rows in the right table to determine whether to output by row list in hash join."},
|
||||
{"create_if_not_exists", false, false, "New setting."},
|
||||
@ -80,6 +81,9 @@ 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"}
|
||||
}
|
||||
},
|
||||
{"24.8",
|
||||
@ -101,9 +105,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",
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Formats/JSONUtils.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -615,28 +616,49 @@ void SerializationArray::serializeTextJSONPretty(const IColumn & column, size_t
|
||||
}
|
||||
|
||||
|
||||
void SerializationArray::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
template <typename ReturnType>
|
||||
ReturnType SerializationArray::deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
deserializeTextImpl(column, istr,
|
||||
[&](IColumn & nested_column)
|
||||
auto deserialize_nested = [&settings, this](IColumn & nested_column, ReadBuffer & buf) -> ReturnType
|
||||
{
|
||||
if constexpr (std::is_same_v<ReturnType, void>)
|
||||
{
|
||||
if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(nested_column))
|
||||
SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(nested_column, istr, settings, nested);
|
||||
SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(nested_column, buf, settings, nested);
|
||||
else
|
||||
nested->deserializeTextJSON(nested_column, istr, settings);
|
||||
}, false);
|
||||
nested->deserializeTextJSON(nested_column, buf, settings);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(nested_column))
|
||||
return SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextJSON(nested_column, buf, settings, nested);
|
||||
return nested->tryDeserializeTextJSON(nested_column, buf, settings);
|
||||
}
|
||||
};
|
||||
|
||||
if (settings.json.empty_as_default)
|
||||
return deserializeTextImpl<ReturnType>(column, istr,
|
||||
[&deserialize_nested, &istr](IColumn & nested_column) -> ReturnType
|
||||
{
|
||||
return JSONUtils::deserializeEmpyStringAsDefaultOrNested<ReturnType>(nested_column, istr, deserialize_nested);
|
||||
}, false);
|
||||
else
|
||||
return deserializeTextImpl<ReturnType>(column, istr,
|
||||
[&deserialize_nested, &istr](IColumn & nested_column) -> ReturnType
|
||||
{
|
||||
return deserialize_nested(nested_column, istr);
|
||||
}, false);
|
||||
}
|
||||
|
||||
|
||||
void SerializationArray::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
deserializeTextJSONImpl<void>(column, istr, settings);
|
||||
}
|
||||
|
||||
bool SerializationArray::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
auto read_nested = [&](IColumn & nested_column)
|
||||
{
|
||||
if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(nested_column))
|
||||
return SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextJSON(nested_column, istr, settings, nested);
|
||||
return nested->tryDeserializeTextJSON(nested_column, istr, settings);
|
||||
};
|
||||
|
||||
return deserializeTextImpl<bool>(column, istr, std::move(read_nested), false);
|
||||
return deserializeTextJSONImpl<bool>(column, istr, settings);
|
||||
}
|
||||
|
||||
|
||||
|
@ -82,6 +82,10 @@ public:
|
||||
SerializationPtr create(const SerializationPtr & prev) const override;
|
||||
ColumnPtr create(const ColumnPtr & prev) const override;
|
||||
};
|
||||
|
||||
private:
|
||||
template <typename ReturnType>
|
||||
ReturnType deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Columns/ColumnMap.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Formats/JSONUtils.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
@ -316,28 +317,52 @@ void SerializationMap::serializeTextJSONPretty(const IColumn & column, size_t ro
|
||||
}
|
||||
|
||||
|
||||
void SerializationMap::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
template <typename ReturnType>
|
||||
ReturnType SerializationMap::deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
deserializeTextImpl(column, istr,
|
||||
[&settings](ReadBuffer & buf, const SerializationPtr & subcolumn_serialization, IColumn & subcolumn)
|
||||
auto deserialize_nested = [&settings](IColumn & subcolumn, ReadBuffer & buf, const SerializationPtr & subcolumn_serialization) -> ReturnType
|
||||
{
|
||||
if constexpr (std::is_same_v<ReturnType, void>)
|
||||
{
|
||||
if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(subcolumn))
|
||||
SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(subcolumn, buf, settings, subcolumn_serialization);
|
||||
else
|
||||
subcolumn_serialization->deserializeTextJSON(subcolumn, buf, settings);
|
||||
});
|
||||
}
|
||||
else
|
||||
{
|
||||
if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(subcolumn))
|
||||
return SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextJSON(subcolumn, buf, settings, subcolumn_serialization);
|
||||
return subcolumn_serialization->tryDeserializeTextJSON(subcolumn, buf, settings);
|
||||
}
|
||||
};
|
||||
|
||||
if (settings.json.empty_as_default)
|
||||
return deserializeTextImpl<ReturnType>(column, istr,
|
||||
[&deserialize_nested](ReadBuffer & buf, const SerializationPtr & subcolumn_serialization, IColumn & subcolumn) -> ReturnType
|
||||
{
|
||||
return JSONUtils::deserializeEmpyStringAsDefaultOrNested<ReturnType>(subcolumn, buf,
|
||||
[&deserialize_nested, &subcolumn_serialization](IColumn & subcolumn_, ReadBuffer & buf_) -> ReturnType
|
||||
{
|
||||
return deserialize_nested(subcolumn_, buf_, subcolumn_serialization);
|
||||
});
|
||||
});
|
||||
else
|
||||
return deserializeTextImpl<ReturnType>(column, istr,
|
||||
[&deserialize_nested](ReadBuffer & buf, const SerializationPtr & subcolumn_serialization, IColumn & subcolumn) -> ReturnType
|
||||
{
|
||||
return deserialize_nested(subcolumn, buf, subcolumn_serialization);
|
||||
});
|
||||
}
|
||||
|
||||
void SerializationMap::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
deserializeTextJSONImpl<void>(column, istr, settings);
|
||||
}
|
||||
|
||||
bool SerializationMap::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
auto reader = [&settings](ReadBuffer & buf, const SerializationPtr & subcolumn_serialization, IColumn & subcolumn)
|
||||
{
|
||||
if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(subcolumn))
|
||||
return SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextJSON(subcolumn, buf, settings, subcolumn_serialization);
|
||||
return subcolumn_serialization->tryDeserializeTextJSON(subcolumn, buf, settings);
|
||||
};
|
||||
|
||||
return deserializeTextImpl<bool>(column, istr, reader);
|
||||
return deserializeTextJSONImpl<bool>(column, istr, settings);
|
||||
}
|
||||
|
||||
void SerializationMap::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
|
@ -74,6 +74,9 @@ private:
|
||||
|
||||
template <typename ReturnType = void, typename Reader>
|
||||
ReturnType deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && reader) const;
|
||||
|
||||
template <typename ReturnType>
|
||||
ReturnType deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Core/Field.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Formats/JSONUtils.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
@ -313,27 +314,9 @@ void SerializationTuple::serializeTextJSONPretty(const IColumn & column, size_t
|
||||
}
|
||||
|
||||
template <typename ReturnType>
|
||||
ReturnType SerializationTuple::deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
ReturnType SerializationTuple::deserializeTupleJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, auto && deserialize_element) const
|
||||
{
|
||||
static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;
|
||||
|
||||
auto deserialize_element = [&](IColumn & element_column, size_t element_pos)
|
||||
{
|
||||
if constexpr (throw_exception)
|
||||
{
|
||||
if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(element_column))
|
||||
SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(element_column, istr, settings, elems[element_pos]);
|
||||
else
|
||||
elems[element_pos]->deserializeTextJSON(element_column, istr, settings);
|
||||
return true;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(element_column))
|
||||
return SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextJSON(element_column, istr, settings, elems[element_pos]);
|
||||
return elems[element_pos]->tryDeserializeTextJSON(element_column, istr, settings);
|
||||
}
|
||||
};
|
||||
static constexpr auto throw_exception = std::is_same_v<ReturnType, void>;
|
||||
|
||||
if (settings.json.read_named_tuples_as_objects
|
||||
&& have_explicit_names)
|
||||
@ -506,12 +489,51 @@ ReturnType SerializationTuple::deserializeTextJSONImpl(IColumn & column, ReadBuf
|
||||
}
|
||||
}
|
||||
|
||||
void SerializationTuple::deserializeTextJSON(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings) const
|
||||
template <typename ReturnType>
|
||||
ReturnType SerializationTuple::deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
deserializeTextJSONImpl(column, istr, settings);
|
||||
auto deserialize_nested = [&settings](IColumn & nested_column, ReadBuffer & buf, const SerializationPtr & nested_column_serialization) -> ReturnType
|
||||
{
|
||||
if constexpr (std::is_same_v<ReturnType, void>)
|
||||
{
|
||||
if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(nested_column))
|
||||
SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(nested_column, buf, settings, nested_column_serialization);
|
||||
else
|
||||
nested_column_serialization->deserializeTextJSON(nested_column, buf, settings);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(nested_column))
|
||||
return SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextJSON(nested_column, buf, settings, nested_column_serialization);
|
||||
else
|
||||
return nested_column_serialization->tryDeserializeTextJSON(nested_column, buf, settings);
|
||||
}
|
||||
};
|
||||
|
||||
if (settings.json.empty_as_default)
|
||||
return deserializeTupleJSONImpl<ReturnType>(column, istr, settings,
|
||||
[&deserialize_nested, &istr, this](IColumn & nested_column, size_t element_pos) -> ReturnType
|
||||
{
|
||||
return JSONUtils::deserializeEmpyStringAsDefaultOrNested<ReturnType>(nested_column, istr,
|
||||
[&deserialize_nested, element_pos, this](IColumn & nested_column_, ReadBuffer & buf) -> ReturnType
|
||||
{
|
||||
return deserialize_nested(nested_column_, buf, elems[element_pos]);
|
||||
});
|
||||
});
|
||||
else
|
||||
return deserializeTupleJSONImpl<ReturnType>(column, istr, settings,
|
||||
[&deserialize_nested, &istr, this](IColumn & nested_column, size_t element_pos) -> ReturnType
|
||||
{
|
||||
return deserialize_nested(nested_column, istr, elems[element_pos]);
|
||||
});
|
||||
}
|
||||
|
||||
bool SerializationTuple::tryDeserializeTextJSON(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings) const
|
||||
void SerializationTuple::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
deserializeTextJSONImpl<void>(column, istr, settings);
|
||||
}
|
||||
|
||||
bool SerializationTuple::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
return deserializeTextJSONImpl<bool>(column, istr, settings);
|
||||
}
|
||||
|
@ -81,7 +81,10 @@ private:
|
||||
template <typename ReturnType = void>
|
||||
ReturnType deserializeTextImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const;
|
||||
|
||||
template <typename ReturnType = void>
|
||||
template <typename ReturnType>
|
||||
ReturnType deserializeTupleJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, auto && deserialize_element) const;
|
||||
|
||||
template <typename ReturnType>
|
||||
ReturnType deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const;
|
||||
|
||||
template <typename ReturnType = void>
|
||||
|
@ -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();
|
||||
|
||||
|
@ -152,6 +152,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se
|
||||
format_settings.json.try_infer_objects_as_tuples = settings.input_format_json_try_infer_named_tuples_from_objects;
|
||||
format_settings.json.throw_on_bad_escape_sequence = settings.input_format_json_throw_on_bad_escape_sequence;
|
||||
format_settings.json.ignore_unnecessary_fields = settings.input_format_json_ignore_unnecessary_fields;
|
||||
format_settings.json.empty_as_default = settings.input_format_json_empty_as_default;
|
||||
format_settings.json.type_json_skip_duplicated_paths = settings.type_json_skip_duplicated_paths;
|
||||
format_settings.null_as_default = settings.input_format_null_as_default;
|
||||
format_settings.force_null_for_omitted_fields = settings.input_format_force_null_for_omitted_fields;
|
||||
|
@ -237,6 +237,7 @@ struct FormatSettings
|
||||
bool infer_incomplete_types_as_strings = true;
|
||||
bool throw_on_bad_escape_sequence = true;
|
||||
bool ignore_unnecessary_fields = true;
|
||||
bool empty_as_default = false;
|
||||
bool type_json_skip_duplicated_paths = false;
|
||||
} json{};
|
||||
|
||||
|
@ -2,12 +2,14 @@
|
||||
#include <Formats/JSONUtils.h>
|
||||
#include <Formats/ReadSchemaUtils.h>
|
||||
#include <Formats/EscapingRuleUtils.h>
|
||||
#include <IO/PeekableReadBuffer.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/WriteBufferValidUTF8.h>
|
||||
#include <DataTypes/Serializations/SerializationNullable.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeObjectDeprecated.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
#include <base/find_symbols.h>
|
||||
|
||||
@ -286,11 +288,19 @@ namespace JSONUtils
|
||||
return true;
|
||||
}
|
||||
|
||||
if (as_nullable)
|
||||
return SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(column, in, format_settings, serialization);
|
||||
auto deserialize = [as_nullable, &format_settings, &serialization](IColumn & column_, ReadBuffer & buf) -> bool
|
||||
{
|
||||
if (as_nullable)
|
||||
return SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(column_, buf, format_settings, serialization);
|
||||
|
||||
serialization->deserializeTextJSON(column, in, format_settings);
|
||||
return true;
|
||||
serialization->deserializeTextJSON(column_, buf, format_settings);
|
||||
return true;
|
||||
};
|
||||
|
||||
if (format_settings.json.empty_as_default)
|
||||
return JSONUtils::deserializeEmpyStringAsDefaultOrNested<bool, false>(column, in, deserialize);
|
||||
else
|
||||
return deserialize(column, in);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
@ -920,6 +930,78 @@ namespace JSONUtils
|
||||
}
|
||||
}
|
||||
|
||||
template <typename ReturnType, bool default_column_return_value>
|
||||
ReturnType deserializeEmpyStringAsDefaultOrNested(IColumn & column, ReadBuffer & istr, const NestedDeserialize<ReturnType> & deserialize_nested)
|
||||
{
|
||||
static constexpr auto throw_exception = std::is_same_v<ReturnType, void>;
|
||||
|
||||
static constexpr auto EMPTY_STRING = "\"\"";
|
||||
static constexpr auto EMPTY_STRING_LENGTH = std::string_view(EMPTY_STRING).length();
|
||||
|
||||
if (istr.eof() || *istr.position() != EMPTY_STRING[0])
|
||||
return deserialize_nested(column, istr);
|
||||
|
||||
auto do_deserialize = [](IColumn & column_, ReadBuffer & buf, auto && check_for_empty_string, auto && deserialize) -> ReturnType
|
||||
{
|
||||
if (check_for_empty_string(buf))
|
||||
{
|
||||
column_.insertDefault();
|
||||
return ReturnType(default_column_return_value);
|
||||
}
|
||||
return deserialize(column_, buf);
|
||||
};
|
||||
|
||||
if (istr.available() >= EMPTY_STRING_LENGTH)
|
||||
{
|
||||
/// We have enough data in buffer to check if we have an empty string.
|
||||
auto check_for_empty_string = [](ReadBuffer & buf) -> bool
|
||||
{
|
||||
auto * pos = buf.position();
|
||||
if (checkString(EMPTY_STRING, buf))
|
||||
return true;
|
||||
buf.position() = pos;
|
||||
return false;
|
||||
};
|
||||
|
||||
return do_deserialize(column, istr, check_for_empty_string, deserialize_nested);
|
||||
}
|
||||
|
||||
/// We don't have enough data in buffer to check if we have an empty string.
|
||||
/// Use PeekableReadBuffer to make a checkpoint before checking for an
|
||||
/// empty string and rollback if check was failed.
|
||||
|
||||
auto check_for_empty_string = [](ReadBuffer & buf) -> bool
|
||||
{
|
||||
auto & peekable_buf = assert_cast<PeekableReadBuffer &>(buf);
|
||||
peekable_buf.setCheckpoint();
|
||||
SCOPE_EXIT(peekable_buf.dropCheckpoint());
|
||||
if (checkString(EMPTY_STRING, peekable_buf))
|
||||
return true;
|
||||
peekable_buf.rollbackToCheckpoint();
|
||||
return false;
|
||||
};
|
||||
|
||||
auto deserialize_nested_with_check = [&deserialize_nested](IColumn & column_, ReadBuffer & buf) -> ReturnType
|
||||
{
|
||||
auto & peekable_buf = assert_cast<PeekableReadBuffer &>(buf);
|
||||
if constexpr (throw_exception)
|
||||
deserialize_nested(column_, peekable_buf);
|
||||
else if (!deserialize_nested(column_, peekable_buf))
|
||||
return ReturnType(false);
|
||||
|
||||
if (unlikely(peekable_buf.hasUnreadData()))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect state while parsing JSON: PeekableReadBuffer has unread data in own memory: {}", String(peekable_buf.position(), peekable_buf.available()));
|
||||
|
||||
return ReturnType(true);
|
||||
};
|
||||
|
||||
PeekableReadBuffer peekable_buf(istr, true);
|
||||
return do_deserialize(column, peekable_buf, check_for_empty_string, deserialize_nested_with_check);
|
||||
}
|
||||
|
||||
template void deserializeEmpyStringAsDefaultOrNested<void, true>(IColumn & column, ReadBuffer & istr, const NestedDeserialize<void> & deserialize_nested);
|
||||
template bool deserializeEmpyStringAsDefaultOrNested<bool, true>(IColumn & column, ReadBuffer & istr, const NestedDeserialize<bool> & deserialize_nested);
|
||||
template bool deserializeEmpyStringAsDefaultOrNested<bool, false>(IColumn & column, ReadBuffer & istr, const NestedDeserialize<bool> & deserialize_nested);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <IO/Progress.h>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <functional>
|
||||
#include <utility>
|
||||
|
||||
namespace DB
|
||||
@ -146,6 +147,16 @@ namespace JSONUtils
|
||||
|
||||
bool skipUntilFieldInObject(ReadBuffer & in, const String & desired_field_name, const FormatSettings::JSON & settings);
|
||||
void skipTheRestOfObject(ReadBuffer & in, const FormatSettings::JSON & settings);
|
||||
|
||||
template <typename ReturnType>
|
||||
using NestedDeserialize = std::function<ReturnType(IColumn &, ReadBuffer &)>;
|
||||
|
||||
template <typename ReturnType, bool default_column_return_value = true>
|
||||
ReturnType deserializeEmpyStringAsDefaultOrNested(IColumn & column, ReadBuffer & istr, const NestedDeserialize<ReturnType> & deserialize_nested);
|
||||
|
||||
extern template void deserializeEmpyStringAsDefaultOrNested<void, true>(IColumn & column, ReadBuffer & istr, const NestedDeserialize<void> & deserialize_nested);
|
||||
extern template bool deserializeEmpyStringAsDefaultOrNested<bool, true>(IColumn & column, ReadBuffer & istr, const NestedDeserialize<bool> & deserialize_nested);
|
||||
extern template bool deserializeEmpyStringAsDefaultOrNested<bool, false>(IColumn & column, ReadBuffer & istr, const NestedDeserialize<bool> & deserialize_nested);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -85,7 +85,9 @@ ConcurrentHashJoin::ConcurrentHashJoin(
|
||||
CurrentMetrics::ConcurrentHashJoinPoolThreads,
|
||||
CurrentMetrics::ConcurrentHashJoinPoolThreadsActive,
|
||||
CurrentMetrics::ConcurrentHashJoinPoolThreadsScheduled,
|
||||
slots))
|
||||
/*max_threads_*/ slots,
|
||||
/*max_free_threads_*/ 0,
|
||||
/*queue_size_*/ slots))
|
||||
, stats_collecting_params(stats_collecting_params_)
|
||||
{
|
||||
hash_joins.resize(slots);
|
||||
|
@ -1236,6 +1236,7 @@ IBlocksStreamPtr HashJoin::getNonJoinedBlocks(const Block & left_sample_block,
|
||||
|
||||
void HashJoin::reuseJoinedData(const HashJoin & join)
|
||||
{
|
||||
have_compressed = join.have_compressed;
|
||||
data = join.data;
|
||||
from_storage_join = true;
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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();
|
||||
|
@ -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;
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <Parsers/ParserShowColumnsQuery.h>
|
||||
|
||||
#include <Parsers/ASTIdentifier_fwd.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTShowColumnsQuery.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
@ -18,7 +18,6 @@ bool ParserShowColumnsQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
||||
ASTPtr from1;
|
||||
ASTPtr from2;
|
||||
|
||||
String from1_str;
|
||||
String from2_str;
|
||||
|
||||
auto query = std::make_shared<ASTShowColumnsQuery>();
|
||||
@ -43,25 +42,18 @@ bool ParserShowColumnsQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
||||
else
|
||||
return false;
|
||||
|
||||
tryGetIdentifierNameInto(from1, from1_str);
|
||||
|
||||
bool abbreviated_form = from1_str.contains("."); // FROM database.table
|
||||
if (abbreviated_form)
|
||||
{
|
||||
std::vector<String> split;
|
||||
boost::split(split, from1_str, boost::is_any_of("."));
|
||||
query->database = split[0];
|
||||
query->table = split[1];
|
||||
}
|
||||
const auto * table_id = from1->as<ASTIdentifier>();
|
||||
if (!table_id)
|
||||
return false;
|
||||
query->table = table_id->shortName();
|
||||
if (table_id->compound())
|
||||
query->database = table_id->name_parts[0];
|
||||
else
|
||||
{
|
||||
if (ParserKeyword(Keyword::FROM).ignore(pos, expected) || ParserKeyword(Keyword::IN).ignore(pos, expected))
|
||||
if (!ParserIdentifier().parse(pos, from2, expected))
|
||||
return false;
|
||||
|
||||
tryGetIdentifierNameInto(from2, from2_str);
|
||||
|
||||
query->table = from1_str;
|
||||
query->database = from2_str;
|
||||
}
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <Parsers/ParserShowIndexesQuery.h>
|
||||
|
||||
#include <Parsers/ASTIdentifier_fwd.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTShowIndexesQuery.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
@ -17,7 +17,6 @@ bool ParserShowIndexesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
||||
ASTPtr from1;
|
||||
ASTPtr from2;
|
||||
|
||||
String from1_str;
|
||||
String from2_str;
|
||||
|
||||
auto query = std::make_shared<ASTShowIndexesQuery>();
|
||||
@ -39,25 +38,18 @@ bool ParserShowIndexesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
||||
else
|
||||
return false;
|
||||
|
||||
tryGetIdentifierNameInto(from1, from1_str);
|
||||
|
||||
bool abbreviated_form = from1_str.contains("."); // FROM database.table
|
||||
if (abbreviated_form)
|
||||
{
|
||||
std::vector<String> split;
|
||||
boost::split(split, from1_str, boost::is_any_of("."));
|
||||
query->database = split[0];
|
||||
query->table = split[1];
|
||||
}
|
||||
const auto * table_id = from1->as<ASTIdentifier>();
|
||||
if (!table_id)
|
||||
return false;
|
||||
query->table = table_id->shortName();
|
||||
if (table_id->compound())
|
||||
query->database = table_id->name_parts[0];
|
||||
else
|
||||
{
|
||||
if (ParserKeyword(Keyword::FROM).ignore(pos, expected) || ParserKeyword(Keyword::IN).ignore(pos, expected))
|
||||
if (!ParserIdentifier().parse(pos, from2, expected))
|
||||
return false;
|
||||
|
||||
tryGetIdentifierNameInto(from2, from2_str);
|
||||
|
||||
query->table = from1_str;
|
||||
query->database = from2_str;
|
||||
}
|
||||
|
||||
|
@ -188,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())
|
||||
@ -269,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");
|
||||
@ -294,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;
|
||||
@ -306,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);
|
||||
}
|
||||
@ -617,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();
|
||||
|
||||
@ -1656,7 +1657,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const
|
||||
merge_parts_query_plan.getCurrentDataStream(),
|
||||
sort_description,
|
||||
partition_key_columns,
|
||||
ctx->merging_params,
|
||||
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,
|
||||
@ -1767,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;
|
||||
|
||||
|
@ -101,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_);
|
||||
@ -111,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);
|
||||
}
|
||||
|
||||
@ -173,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};
|
||||
@ -211,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};
|
||||
@ -223,12 +225,6 @@ private:
|
||||
/// Proper initialization is responsibility of the author
|
||||
struct ExecuteAndFinalizeHorizontalPartRuntimeContext : public IStageRuntimeContext
|
||||
{
|
||||
/// Dependencies
|
||||
String suffix;
|
||||
bool need_prefix;
|
||||
MergeTreeData::MergingParams merging_params{};
|
||||
|
||||
DiskPtr disk{nullptr};
|
||||
bool need_remove_expired_values{false};
|
||||
bool force_ttl{false};
|
||||
CompressionCodecPtr compression_codec{nullptr};
|
||||
@ -263,7 +259,6 @@ private:
|
||||
|
||||
using ExecuteAndFinalizeHorizontalPartRuntimeContextPtr = std::shared_ptr<ExecuteAndFinalizeHorizontalPartRuntimeContext>;
|
||||
|
||||
|
||||
struct ExecuteAndFinalizeHorizontalPart : public IStage
|
||||
{
|
||||
bool execute() override;
|
||||
@ -352,7 +347,6 @@ private:
|
||||
|
||||
using VerticalMergeRuntimeContextPtr = std::shared_ptr<VerticalMergeRuntimeContext>;
|
||||
|
||||
|
||||
struct VerticalMergeStage : public IStage
|
||||
{
|
||||
bool execute() override;
|
||||
|
@ -15,6 +15,7 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int SUPPORT_IS_DISABLED;
|
||||
extern const int REPLICA_STATUS_CHANGED;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
ReplicatedMergeTreeAttachThread::ReplicatedMergeTreeAttachThread(StorageReplicatedMergeTree & storage_)
|
||||
@ -117,6 +118,67 @@ void ReplicatedMergeTreeAttachThread::checkHasReplicaMetadataInZooKeeper(const z
|
||||
}
|
||||
}
|
||||
|
||||
Int32 ReplicatedMergeTreeAttachThread::fixReplicaMetadataVersionIfNeeded(zkutil::ZooKeeperPtr zookeeper)
|
||||
{
|
||||
const String & zookeeper_path = storage.zookeeper_path;
|
||||
const String & replica_path = storage.replica_path;
|
||||
const bool replica_readonly = storage.is_readonly;
|
||||
|
||||
for (size_t i = 0; i != 2; ++i)
|
||||
{
|
||||
String replica_metadata_version_str;
|
||||
const bool replica_metadata_version_exists = zookeeper->tryGet(replica_path + "/metadata_version", replica_metadata_version_str);
|
||||
if (!replica_metadata_version_exists)
|
||||
return -1;
|
||||
|
||||
const Int32 metadata_version = parse<Int32>(replica_metadata_version_str);
|
||||
|
||||
if (metadata_version != 0 || replica_readonly)
|
||||
{
|
||||
/// No need to fix anything
|
||||
return metadata_version;
|
||||
}
|
||||
|
||||
Coordination::Stat stat;
|
||||
zookeeper->get(fs::path(zookeeper_path) / "metadata", &stat);
|
||||
if (stat.version == 0)
|
||||
{
|
||||
/// No need to fix anything
|
||||
return metadata_version;
|
||||
}
|
||||
|
||||
ReplicatedMergeTreeQueue & queue = storage.queue;
|
||||
queue.pullLogsToQueue(zookeeper);
|
||||
if (queue.getStatus().metadata_alters_in_queue != 0)
|
||||
{
|
||||
LOG_DEBUG(log, "No need to update metadata_version as there are ALTER_METADATA entries in the queue");
|
||||
return metadata_version;
|
||||
}
|
||||
|
||||
const Coordination::Requests ops = {
|
||||
zkutil::makeSetRequest(fs::path(replica_path) / "metadata_version", std::to_string(stat.version), 0),
|
||||
zkutil::makeCheckRequest(fs::path(zookeeper_path) / "metadata", stat.version),
|
||||
};
|
||||
Coordination::Responses ops_responses;
|
||||
const auto code = zookeeper->tryMulti(ops, ops_responses);
|
||||
if (code == Coordination::Error::ZOK)
|
||||
{
|
||||
LOG_DEBUG(log, "Successfully set metadata_version to {}", stat.version);
|
||||
return stat.version;
|
||||
}
|
||||
if (code != Coordination::Error::ZBADVERSION)
|
||||
{
|
||||
throw zkutil::KeeperException(code);
|
||||
}
|
||||
}
|
||||
|
||||
/// Second attempt is only possible if metadata_version != 0 or metadata.version changed during the first attempt.
|
||||
/// If metadata_version != 0, on second attempt we will return the new metadata_version.
|
||||
/// If metadata.version changed, on second attempt we will either get metadata_version != 0 and return the new metadata_version or we will get metadata_alters_in_queue != 0 and return 0.
|
||||
/// Either way, on second attempt this method should return.
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to fix replica metadata_version in ZooKeeper after two attempts");
|
||||
}
|
||||
|
||||
void ReplicatedMergeTreeAttachThread::runImpl()
|
||||
{
|
||||
storage.setZooKeeper();
|
||||
@ -160,11 +222,11 @@ void ReplicatedMergeTreeAttachThread::runImpl()
|
||||
/// Just in case it was not removed earlier due to connection loss
|
||||
zookeeper->tryRemove(replica_path + "/flags/force_restore_data");
|
||||
|
||||
String replica_metadata_version;
|
||||
const bool replica_metadata_version_exists = zookeeper->tryGet(replica_path + "/metadata_version", replica_metadata_version);
|
||||
const Int32 replica_metadata_version = fixReplicaMetadataVersionIfNeeded(zookeeper);
|
||||
const bool replica_metadata_version_exists = replica_metadata_version != -1;
|
||||
if (replica_metadata_version_exists)
|
||||
{
|
||||
storage.setInMemoryMetadata(metadata_snapshot->withMetadataVersion(parse<int>(replica_metadata_version)));
|
||||
storage.setInMemoryMetadata(metadata_snapshot->withMetadataVersion(replica_metadata_version));
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -48,6 +48,8 @@ private:
|
||||
void runImpl();
|
||||
|
||||
void finalizeInitialization();
|
||||
|
||||
Int32 fixReplicaMetadataVersionIfNeeded(zkutil::ZooKeeperPtr zookeeper);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -2222,6 +2222,7 @@ ReplicatedMergeTreeQueue::Status ReplicatedMergeTreeQueue::getStatus() const
|
||||
res.inserts_in_queue = 0;
|
||||
res.merges_in_queue = 0;
|
||||
res.part_mutations_in_queue = 0;
|
||||
res.metadata_alters_in_queue = 0;
|
||||
res.queue_oldest_time = 0;
|
||||
res.inserts_oldest_time = 0;
|
||||
res.merges_oldest_time = 0;
|
||||
@ -2264,6 +2265,11 @@ ReplicatedMergeTreeQueue::Status ReplicatedMergeTreeQueue::getStatus() const
|
||||
res.oldest_part_to_mutate_to = entry->new_part_name;
|
||||
}
|
||||
}
|
||||
|
||||
if (entry->type == LogEntry::ALTER_METADATA)
|
||||
{
|
||||
++res.metadata_alters_in_queue;
|
||||
}
|
||||
}
|
||||
|
||||
return res;
|
||||
|
@ -473,6 +473,7 @@ public:
|
||||
UInt32 inserts_in_queue;
|
||||
UInt32 merges_in_queue;
|
||||
UInt32 part_mutations_in_queue;
|
||||
UInt32 metadata_alters_in_queue;
|
||||
UInt32 queue_oldest_time;
|
||||
UInt32 inserts_oldest_time;
|
||||
UInt32 merges_oldest_time;
|
||||
|
@ -277,7 +277,7 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs(
|
||||
|
||||
if (has_valid_arguments)
|
||||
{
|
||||
if (is_replicated_database && local_context->getSettingsRef().database_replicated_allow_replicated_engine_arguments == 0)
|
||||
if (!query.attach && is_replicated_database && local_context->getSettingsRef().database_replicated_allow_replicated_engine_arguments == 0)
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"It's not allowed to specify explicit zookeeper_path and replica_name "
|
||||
@ -285,7 +285,7 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs(
|
||||
"specify them explicitly, enable setting "
|
||||
"database_replicated_allow_replicated_engine_arguments.");
|
||||
}
|
||||
else if (is_replicated_database && local_context->getSettingsRef().database_replicated_allow_replicated_engine_arguments == 1)
|
||||
else if (!query.attach && is_replicated_database && local_context->getSettingsRef().database_replicated_allow_replicated_engine_arguments == 1)
|
||||
{
|
||||
LOG_WARNING(&Poco::Logger::get("registerStorageMergeTree"), "It's not recommended to explicitly specify "
|
||||
"zookeeper_path and replica_name in ReplicatedMergeTree arguments");
|
||||
@ -305,7 +305,7 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs(
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica name must be a string literal{}", verbose_help_message);
|
||||
|
||||
|
||||
if (is_replicated_database && local_context->getSettingsRef().database_replicated_allow_replicated_engine_arguments == 2)
|
||||
if (!query.attach && is_replicated_database && local_context->getSettingsRef().database_replicated_allow_replicated_engine_arguments == 2)
|
||||
{
|
||||
LOG_WARNING(&Poco::Logger::get("registerStorageMergeTree"), "Replacing user-provided ZooKeeper path and replica name ({}, {}) "
|
||||
"with default arguments", zookeeper_path, replica_name);
|
||||
|
@ -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);
|
||||
|
@ -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_,
|
||||
|
@ -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_);
|
||||
|
@ -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_,
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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,
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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>;
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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()
|
||||
|
@ -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);
|
||||
|
@ -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)
|
||||
|
@ -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 &);
|
||||
|
||||
|
@ -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:
|
||||
|
@ -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>;
|
||||
|
@ -1549,3 +1549,19 @@ def test_all_groups_cluster(started_cluster):
|
||||
assert "bad_settings_node\ndummy_node\n" == bad_settings_node.query(
|
||||
"select host_name from system.clusters where name='all_groups.db_cluster' order by host_name"
|
||||
)
|
||||
|
||||
|
||||
def test_detach_attach_table(started_cluster):
|
||||
main_node.query("DROP DATABASE IF EXISTS detach_attach_db SYNC")
|
||||
main_node.query(
|
||||
"CREATE DATABASE detach_attach_db ENGINE = Replicated('/clickhouse/databases/detach_attach_db');"
|
||||
)
|
||||
main_node.query(
|
||||
"CREATE TABLE detach_attach_db.detach_attach_table (k UInt64) ENGINE=ReplicatedMergeTree ORDER BY k;"
|
||||
)
|
||||
main_node.query("INSERT INTO detach_attach_db.detach_attach_table VALUES (1);")
|
||||
main_node.query("DETACH TABLE detach_attach_db.detach_attach_table PERMANENTLY;")
|
||||
main_node.query("ATTACH TABLE detach_attach_db.detach_attach_table;")
|
||||
assert (
|
||||
main_node.query("SELECT * FROM detach_attach_db.detach_attach_table;") == "1\n"
|
||||
)
|
||||
|
@ -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>
|
||||
|
@ -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
|
||||
|
@ -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-----
|
@ -0,0 +1 @@
|
||||
subjectAltName=URI:spiffe://bar.com/foo/boo/far
|
@ -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-----
|
@ -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-----
|
@ -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-----
|
@ -0,0 +1 @@
|
||||
subjectAltName=URI:spiffe://bar.com/foo/baz/far
|
@ -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-----
|
@ -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-----
|
@ -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"
|
||||
|
@ -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>
|
||||
|
@ -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")
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -45,3 +45,7 @@ uint64 UInt64 NO PRI SOR \N
|
||||
int32 Int32 NO \N
|
||||
str String NO \N
|
||||
uint64 UInt64 NO PRI SOR \N
|
||||
--- SHOW COLUMNS FROM table with dots
|
||||
int32 Nullable(Int32) YES \N
|
||||
str String NO SOR \N
|
||||
uint64 UInt64 NO PRI SOR \N
|
||||
|
@ -90,3 +90,18 @@ SHOW COLUMNS FROM database_123456789abcde.tab;
|
||||
DROP DATABASE database_123456789abcde;
|
||||
|
||||
DROP TABLE tab;
|
||||
|
||||
DROP TABLE IF EXISTS `tab.with.dots`;
|
||||
CREATE TABLE `tab.with.dots`
|
||||
(
|
||||
`uint64` UInt64,
|
||||
`int32` Nullable(Int32) COMMENT 'example comment',
|
||||
`str` String,
|
||||
INDEX idx str TYPE set(1000)
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
PRIMARY KEY (uint64)
|
||||
ORDER BY (uint64, str);
|
||||
SELECT '--- SHOW COLUMNS FROM table with dots';
|
||||
SHOW COLUMNS FROM `tab.with.dots`;
|
||||
DROP TABLE `tab.with.dots`;
|
||||
|
@ -49,3 +49,10 @@ tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES
|
||||
--- Short form
|
||||
tbl 1 mmi_idx 1 \N 0 \N \N \N MINMAX YES b
|
||||
tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES
|
||||
--- SHOW INDEX FROM table with dots
|
||||
tab.with.dots 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b
|
||||
tab.with.dots 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d
|
||||
tab.with.dots 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e
|
||||
tab.with.dots 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES
|
||||
tab.with.dots 1 PRIMARY 2 a A 0 \N \N \N PRIMARY YES
|
||||
tab.with.dots 1 set_idx 1 \N 0 \N \N \N SET YES e
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user