Merge branch 'master' into fix-use-concurrency-control

This commit is contained in:
serxa 2024-09-16 15:05:14 +00:00
commit d86ad992f1
212 changed files with 4757 additions and 976 deletions

View File

@ -4,15 +4,31 @@ description: Prints workflow debug info
runs:
using: "composite"
steps:
- name: Print envs
- name: Envs, event.json and contexts
shell: bash
run: |
echo "::group::Envs"
env
echo "::endgroup::"
- name: Print Event.json
shell: bash
run: |
echo "::group::Event.json"
echo '::group::Environment variables'
env | sort
echo '::endgroup::'
echo '::group::event.json'
python3 -m json.tool "$GITHUB_EVENT_PATH"
echo "::endgroup::"
echo '::endgroup::'
cat << 'EOF'
::group::github context
${{ toJSON(github) }}
::endgroup::
::group::env context
${{ toJSON(env) }}
::endgroup::
::group::runner context
${{ toJSON(runner) }}
::endgroup::
::group::job context
${{ toJSON(job) }}
::endgroup::
EOF

View File

@ -27,6 +27,8 @@ jobs:
clear-repository: true # to ensure correct digests
fetch-depth: 0 # to get version
filter: tree:0
- name: Debug Info
uses: ./.github/actions/debug
- name: Labels check
run: |
cd "$GITHUB_WORKSPACE/tests/ci"

View File

@ -33,6 +33,8 @@ jobs:
clear-repository: true
token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}}
fetch-depth: 0
- name: Debug Info
uses: ./.github/actions/debug
- name: Cherry pick
run: |
cd "$GITHUB_WORKSPACE/tests/ci"

View File

@ -56,13 +56,13 @@ jobs:
GH_TOKEN: ${{ secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN }}
runs-on: [self-hosted, release-maker]
steps:
- name: DebugInfo
uses: hmarr/debug-action@f7318c783045ac39ed9bb497e22ce835fdafbfe6
- name: Check out repository code
uses: ClickHouse/checkout@v1
with:
token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}}
fetch-depth: 0
- name: Debug Info
uses: ./.github/actions/debug
- name: Prepare Release Info
shell: bash
run: |

View File

@ -11,6 +11,7 @@ name: Build docker images
required: false
type: boolean
default: false
jobs:
DockerBuildAarch64:
runs-on: [self-hosted, style-checker-aarch64]

View File

@ -8,20 +8,21 @@ on: # yamllint disable-line rule:truthy
schedule:
- cron: '0 */6 * * *'
workflow_dispatch:
jobs:
RunConfig:
runs-on: [self-hosted, style-checker-aarch64]
outputs:
data: ${{ steps.runconfig.outputs.CI_DATA }}
steps:
- name: DebugInfo
uses: hmarr/debug-action@f7318c783045ac39ed9bb497e22ce835fdafbfe6
- name: Check out repository code
uses: ClickHouse/checkout@v1
with:
clear-repository: true # to ensure correct digests
fetch-depth: 0 # to get version
filter: tree:0
- name: Debug Info
uses: ./.github/actions/debug
- name: PrepareRunConfig
id: runconfig
run: |

View File

@ -15,14 +15,14 @@ jobs:
outputs:
data: ${{ steps.runconfig.outputs.CI_DATA }}
steps:
- name: DebugInfo
uses: hmarr/debug-action@f7318c783045ac39ed9bb497e22ce835fdafbfe6
- name: Check out repository code
uses: ClickHouse/checkout@v1
with:
clear-repository: true # to ensure correct digests
fetch-depth: 0 # to get version
filter: tree:0
- name: Debug Info
uses: ./.github/actions/debug
- name: Merge sync PR
run: |
cd "$GITHUB_WORKSPACE/tests/ci"

View File

@ -14,14 +14,14 @@ jobs:
outputs:
data: ${{ steps.runconfig.outputs.CI_DATA }}
steps:
- name: DebugInfo
uses: hmarr/debug-action@f7318c783045ac39ed9bb497e22ce835fdafbfe6
- name: Check out repository code
uses: ClickHouse/checkout@v1
with:
clear-repository: true # to ensure correct digests
fetch-depth: 0 # to get a version
filter: tree:0
- name: Debug Info
uses: ./.github/actions/debug
- name: Cancel PR workflow
run: |
python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --cancel-previous-run

View File

@ -15,14 +15,14 @@ jobs:
outputs:
data: ${{ steps.runconfig.outputs.CI_DATA }}
steps:
- name: DebugInfo
uses: hmarr/debug-action@f7318c783045ac39ed9bb497e22ce835fdafbfe6
- name: Check out repository code
uses: ClickHouse/checkout@v1
with:
clear-repository: true # to ensure correct digests
fetch-depth: 0 # to get version
filter: tree:0
- name: Debug Info
uses: ./.github/actions/debug
- name: PrepareRunConfig
id: runconfig
run: |

View File

@ -25,14 +25,14 @@ jobs:
outputs:
data: ${{ steps.runconfig.outputs.CI_DATA }}
steps:
- name: DebugInfo
uses: hmarr/debug-action@f7318c783045ac39ed9bb497e22ce835fdafbfe6
- name: Check out repository code
uses: ClickHouse/checkout@v1
with:
clear-repository: true # to ensure correct digests
fetch-depth: 0 # to get a version
filter: tree:0
- name: Debug Info
uses: ./.github/actions/debug
- name: Cancel previous Sync PR workflow
run: |
python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --cancel-previous-run

View File

@ -24,6 +24,8 @@ jobs:
clear-repository: true # to ensure correct digests
fetch-depth: 0 # to get version
filter: tree:0
- name: Debug Info
uses: ./.github/actions/debug
- name: Labels check
run: |
cd "$GITHUB_WORKSPACE/tests/ci"

View File

@ -62,8 +62,6 @@ jobs:
env:
GITHUB_JOB_OVERRIDDEN: ${{inputs.test_name}}
steps:
- name: DebugInfo
uses: hmarr/debug-action@f7318c783045ac39ed9bb497e22ce835fdafbfe6
- name: Check out repository code
uses: ClickHouse/checkout@v1
with:
@ -72,6 +70,8 @@ jobs:
submodules: ${{inputs.submodules}}
fetch-depth: ${{inputs.checkout_depth}}
filter: tree:0
- name: Debug Info
uses: ./.github/actions/debug
- name: Set build envs
run: |
cat >> "$GITHUB_ENV" << 'EOF'

View File

@ -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}>

View File

@ -13,16 +13,17 @@ Here is a complete list of available database engines. Follow the links for more
- [Atomic](../../engines/database-engines/atomic.md)
- [MySQL](../../engines/database-engines/mysql.md)
- [Lazy](../../engines/database-engines/lazy.md)
- [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md)
- [MaterializedMySQL](../../engines/database-engines/materialized-mysql.md)
- [Lazy](../../engines/database-engines/lazy.md)
- [MySQL](../../engines/database-engines/mysql.md)
- [PostgreSQL](../../engines/database-engines/postgresql.md)
- [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md)
- [Replicated](../../engines/database-engines/replicated.md)
- [SQLite](../../engines/database-engines/sqlite.md)

View File

@ -107,6 +107,10 @@ The vector similarity index currently does not work with per-table, non-default
[here](https://github.com/ClickHouse/ClickHouse/pull/51325#issuecomment-1605920475)). If necessary, the value must be changed in config.xml.
:::
Vector index creation is known to be slow. To speed the process up, index creation can be parallelized. The maximum number of threads can be
configured using server configuration
setting [max_build_vector_similarity_index_thread_pool_size](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters_max_build_vector_similarity_index_thread_pool_size).
ANN indexes are built during column insertion and merge. As a result, `INSERT` and `OPTIMIZE` statements will be slower than for ordinary
tables. ANNIndexes are ideally used only with immutable or rarely changed data, respectively when are far more read requests than write
requests.

View File

@ -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`.

View File

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

View File

@ -491,6 +491,14 @@ Type: Double
Default: 0.9
## max_build_vector_similarity_index_thread_pool_size {#server_configuration_parameters_max_build_vector_similarity_index_thread_pool_size}
The maximum number of threads to use for building vector indexes. 0 means all cores.
Type: UInt64
Default: 16
## cgroups_memory_usage_observer_wait_time
Interval in seconds during which the server's maximum allowed memory consumption is adjusted by the corresponding threshold in cgroups. (see

View File

@ -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}

View File

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

View File

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

View File

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

View File

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

View File

@ -178,6 +178,9 @@
M(ObjectStorageAzureThreads, "Number of threads in the AzureObjectStorage thread pool.") \
M(ObjectStorageAzureThreadsActive, "Number of threads in the AzureObjectStorage thread pool running a task.") \
M(ObjectStorageAzureThreadsScheduled, "Number of queued or active jobs in the AzureObjectStorage thread pool.") \
M(BuildVectorSimilarityIndexThreads, "Number of threads in the build vector similarity index thread pool.") \
M(BuildVectorSimilarityIndexThreadsActive, "Number of threads in the build vector similarity index thread pool running a task.") \
M(BuildVectorSimilarityIndexThreadsScheduled, "Number of queued or active jobs in the build vector similarity index thread pool.") \
\
M(DiskPlainRewritableAzureDirectoryMapSize, "Number of local-to-remote path entries in the 'plain_rewritable' in-memory map for AzureObjectStorage.") \
M(DiskPlainRewritableLocalDirectoryMapSize, "Number of local-to-remote path entries in the 'plain_rewritable' in-memory map for LocalObjectStorage.") \

View File

@ -609,6 +609,7 @@
M(728, UNEXPECTED_DATA_TYPE) \
M(729, ILLEGAL_TIME_SERIES_TAGS) \
M(730, REFRESH_FAILED) \
M(731, QUERY_CACHE_USED_WITH_NON_THROW_OVERFLOW_MODE) \
\
M(900, DISTRIBUTED_CACHE_ERROR) \
M(901, CANNOT_USE_DISTRIBUTED_CACHE) \

View File

@ -63,6 +63,7 @@ static struct InitFiu
REGULAR(keepermap_fail_drop_data) \
REGULAR(lazy_pipe_fds_fail_close) \
PAUSEABLE(infinite_sleep) \
PAUSEABLE(stop_moving_part_before_swap_with_active) \
namespace FailPoints

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -50,7 +50,7 @@ namespace DB
M(UInt32, asynchronous_heavy_metrics_update_period_s, 120, "Period in seconds for updating heavy asynchronous metrics.", 0) \
M(String, default_database, "default", "Default database name.", 0) \
M(String, tmp_policy, "", "Policy for storage with temporary data.", 0) \
M(UInt64, max_temporary_data_on_disk_size, 0, "The maximum amount of storage that could be used for external aggregation, joins or sorting., ", 0) \
M(UInt64, max_temporary_data_on_disk_size, 0, "The maximum amount of storage that could be used for external aggregation, joins or sorting.", 0) \
M(String, temporary_data_in_cache, "", "Cache disk name for temporary data.", 0) \
M(UInt64, aggregate_function_group_array_max_element_size, 0xFFFFFF, "Max array element size in bytes for groupArray function. This limit is checked at serialization and help to avoid large state size.", 0) \
M(GroupArrayActionWhenLimitReached, aggregate_function_group_array_action_when_limit_is_reached, GroupArrayActionWhenLimitReached::THROW, "Action to execute when max array element size is exceeded in groupArray: `throw` exception, or `discard` extra values", 0) \
@ -65,6 +65,7 @@ namespace DB
M(UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0) \
M(Bool, async_insert_queue_flush_on_shutdown, true, "If true queue of asynchronous inserts is flushed on graceful shutdown", 0) \
M(Bool, ignore_empty_sql_security_in_create_view_query, true, "If true, ClickHouse doesn't write defaults for empty SQL security statement in CREATE VIEW queries. This setting is only necessary for the migration period and will become obsolete in 24.4", 0) \
M(UInt64, max_build_vector_similarity_index_thread_pool_size, 16, "The maximum number of threads to use to build vector similarity indexes. 0 means all cores.", 0) \
\
/* Database Catalog */ \
M(UInt64, database_atomic_delay_before_drop_table_sec, 8 * 60, "The delay during which a dropped table can be restored using the UNDROP statement. If DROP TABLE ran with a SYNC modifier, the setting is ignored.", 0) \

View File

@ -923,6 +923,9 @@ class IColumn;
M(Bool, implicit_transaction, false, "If enabled and not already inside a transaction, wraps the query inside a full transaction (begin + commit or rollback)", 0) \
M(UInt64, grace_hash_join_initial_buckets, 1, "Initial number of grace hash join buckets", 0) \
M(UInt64, grace_hash_join_max_buckets, 1024, "Limit on the number of grace hash join buckets", 0) \
M(Int32, join_to_sort_minimum_perkey_rows, 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", 0) \
M(Int32, join_to_sort_maximum_table_rows, 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.", 0) \
M(Bool, allow_experimental_join_right_table_sorting, 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.", 0) \
M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \
M(Bool, use_hive_partitioning, false, "Allows to use hive partitioning for File, URL, S3, AzureBlobStorage and HDFS engines.", 0)\
\
@ -945,6 +948,7 @@ class IColumn;
M(Bool, parallel_replicas_prefer_local_join, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN.", 0) \
M(UInt64, parallel_replicas_mark_segment_size, 128, "Parts virtually divided into segments to be distributed between replicas for parallel reading. This setting controls the size of these segments. Not recommended to change until you're absolutely sure in what you're doing", 0) \
M(Bool, allow_archive_path_syntax, true, "File/S3 engines/table function will parse paths with '::' as '<archive> :: <file>' if archive has correct extension", 0) \
M(Bool, parallel_replicas_local_plan, false, "Build local plan for local replica", 0) \
\
M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental inverted index.", 0) \
M(Bool, allow_experimental_full_text_index, false, "If it is set to true, allow to use experimental full-text index.", 0) \
@ -1140,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) \

View File

@ -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."},
@ -79,6 +80,10 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"output_format_identifier_quoting_style", "Backticks", "Backticks", "New setting."},
{"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",
@ -99,7 +104,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"use_json_alias_for_old_object_type", true, false, "Use JSON type alias to create new JSON type"},
{"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"}
{"input_format_try_infer_datetimes_only_datetime64", true, false, "Allow to infer DateTime instead of DateTime64 in data formats"},
}
},
{"24.7",

View File

@ -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);
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);
}

View File

@ -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;
};
}

View File

@ -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);
});
}
bool SerializationMap::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
auto reader = [&settings](ReadBuffer & buf, const SerializationPtr & subcolumn_serialization, IColumn & subcolumn)
}
else
{
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);
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
{
return deserializeTextJSONImpl<bool>(column, istr, settings);
}
void SerializationMap::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const

View File

@ -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;
};
}

View File

@ -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);
}

View File

@ -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>

View File

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

View File

@ -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;

View File

@ -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{};

View File

@ -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;
}
auto deserialize = [as_nullable, &format_settings, &serialization](IColumn & column_, ReadBuffer & buf) -> bool
{
if (as_nullable)
return SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(column, in, format_settings, serialization);
return SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(column_, buf, format_settings, serialization);
serialization->deserializeTextJSON(column, in, format_settings);
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);
}
}

View File

@ -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);
}
}

View File

@ -50,13 +50,6 @@ private:
return executeNonconstant(input);
}
[[maybe_unused]] String toString() const
{
WriteBufferFromOwnString buf;
buf << "format:" << format << ", rows:" << rows << ", is_literal:" << is_literal << ", input:" << input.dumpStructure() << "\n";
return buf.str();
}
private:
ColumnWithTypeAndName executeLiteral(std::string_view literal) const
{
@ -231,9 +224,7 @@ public:
const auto & instruction = instructions[i];
try
{
// std::cout << "instruction[" << i << "]:" << instructions[i].toString() << std::endl;
concat_args[i] = instruction.execute();
// std::cout << "concat_args[" << i << "]:" << concat_args[i].dumpStructure() << std::endl;
}
catch (const fmt::v9::format_error & e)
{
@ -358,7 +349,14 @@ private:
REGISTER_FUNCTION(Printf)
{
factory.registerFunction<FunctionPrintf>();
factory.registerFunction<FunctionPrintf>(
FunctionDocumentation{.description=R"(
The `printf` function formats the given string with the values (strings, integers, floating-points etc.) listed in the arguments, similar to printf function in C++.
The format string can contain format specifiers starting with `%` character.
Anything not contained in `%` and the following format specifier is considered literal text and copied verbatim into the output.
Literal `%` character can be escaped by `%%`.)", .examples{{"sum", "select printf('%%%s %s %d', 'Hello', 'World', 2024);", "%Hello World 2024"}}, .categories{"String"}
});
}
}

View File

@ -28,6 +28,7 @@
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/StorageSnapshot.h>
#include <Storages/buildQueryTreeForShard.h>
#include <Processors/QueryPlan/ParallelReplicasLocalPlan.h>
#include <Storages/getStructureOfRemoteTable.h>
namespace DB
@ -38,6 +39,7 @@ namespace ErrorCodes
extern const int TOO_LARGE_DISTRIBUTED_DEPTH;
extern const int LOGICAL_ERROR;
extern const int UNEXPECTED_CLUSTER;
extern const int INCONSISTENT_CLUSTER_DEFINITION;
}
namespace ClusterProxy
@ -439,7 +441,8 @@ void executeQueryWithParallelReplicas(
QueryProcessingStage::Enum processed_stage,
const ASTPtr & query_ast,
ContextPtr context,
std::shared_ptr<const StorageLimitsList> storage_limits)
std::shared_ptr<const StorageLimitsList> storage_limits,
QueryPlanStepPtr analyzed_read_from_merge_tree)
{
auto logger = getLogger("executeQueryWithParallelReplicas");
LOG_DEBUG(logger, "Executing read from {}, header {}, query ({}), stage {} with parallel replicas",
@ -485,12 +488,12 @@ void executeQueryWithParallelReplicas(
shard_num = column->getUInt(0);
}
const auto shard_count = not_optimized_cluster->getShardCount();
ClusterPtr new_cluster = not_optimized_cluster;
/// if got valid shard_num from query initiator, then parallel replicas scope is the specified shard
/// shards are numbered in order of appearance in the cluster config
if (shard_num > 0)
{
const auto shard_count = not_optimized_cluster->getShardCount();
if (shard_num > shard_count)
throw Exception(
ErrorCodes::LOGICAL_ERROR,
@ -516,11 +519,99 @@ void executeQueryWithParallelReplicas(
"`cluster_for_parallel_replicas` setting refers to cluster with several shards. Expected a cluster with one shard");
}
const auto & shard = new_cluster->getShardsInfo().at(0);
size_t max_replicas_to_use = settings.max_parallel_replicas;
if (max_replicas_to_use > shard.getAllNodeCount())
{
LOG_INFO(
getLogger("ReadFromParallelRemoteReplicasStep"),
"The number of replicas requested ({}) is bigger than the real number available in the cluster ({}). "
"Will use the latter number to execute the query.",
settings.max_parallel_replicas,
shard.getAllNodeCount());
max_replicas_to_use = shard.getAllNodeCount();
}
auto coordinator = std::make_shared<ParallelReplicasReadingCoordinator>(max_replicas_to_use, settings.parallel_replicas_mark_segment_size);
auto external_tables = new_context->getExternalTables();
std::vector<ConnectionPoolWithFailover::Base::ShuffledPool> shuffled_pool;
if (max_replicas_to_use < shard.getAllNodeCount())
{
// will be shuffled according to `load_balancing` setting
shuffled_pool = shard.pool->getShuffledPools(settings);
}
else
{
/// If all replicas in cluster are used for query execution,
/// try to preserve replicas order as in cluster definition.
/// It's important for data locality during query execution
/// independently of the query initiator
auto priority_func = [](size_t i) { return Priority{static_cast<Int64>(i)}; };
shuffled_pool = shard.pool->getShuffledPools(settings, priority_func);
}
std::vector<ConnectionPoolPtr> pools_to_use;
pools_to_use.reserve(shuffled_pool.size());
for (auto & pool : shuffled_pool)
pools_to_use.emplace_back(std::move(pool.pool));
/// do not build local plan for distributed queries for now (address it later)
if (settings.allow_experimental_analyzer && settings.parallel_replicas_local_plan && !shard_num)
{
/// find local replica index in pool
std::optional<size_t> local_replica_index;
for (size_t i = 0, s = pools_to_use.size(); i < s; ++i)
{
const auto & hostname = pools_to_use[i]->getHost();
const auto found = std::find_if(
begin(shard.local_addresses),
end(shard.local_addresses),
[&hostname](const Cluster::Address & local_addr) { return hostname == local_addr.host_name; });
if (found != shard.local_addresses.end())
{
local_replica_index = i;
break;
}
}
if (!local_replica_index)
throw Exception(
ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION,
"Local replica is not found in '{}' cluster definition, see 'cluster_for_parallel_replicas' setting",
new_cluster->getName());
// resize the pool but keep local replicas in it (and update its index)
chassert(max_replicas_to_use <= pools_to_use.size());
if (local_replica_index >= max_replicas_to_use)
{
std::swap(pools_to_use[max_replicas_to_use - 1], pools_to_use[local_replica_index.value()]);
local_replica_index = max_replicas_to_use - 1;
}
pools_to_use.resize(max_replicas_to_use);
auto [local_plan, with_parallel_replicas] = createLocalPlanForParallelReplicas(
query_ast,
header,
new_context,
processed_stage,
coordinator,
std::move(analyzed_read_from_merge_tree),
local_replica_index.value());
if (!with_parallel_replicas)
{
query_plan = std::move(*local_plan);
return;
}
LOG_DEBUG(logger, "Local replica got replica number {}", local_replica_index.value());
auto read_from_remote = std::make_unique<ReadFromParallelRemoteReplicasStep>(
query_ast,
new_cluster,
storage_id,
coordinator,
header,
processed_stage,
new_context,
@ -528,9 +619,47 @@ void executeQueryWithParallelReplicas(
std::move(scalars),
std::move(external_tables),
getLogger("ReadFromParallelRemoteReplicasStep"),
std::move(storage_limits));
std::move(storage_limits),
std::move(pools_to_use),
local_replica_index);
auto remote_plan = std::make_unique<QueryPlan>();
remote_plan->addStep(std::move(read_from_remote));
DataStreams input_streams;
input_streams.reserve(2);
input_streams.emplace_back(local_plan->getCurrentDataStream());
input_streams.emplace_back(remote_plan->getCurrentDataStream());
std::vector<QueryPlanPtr> plans;
plans.emplace_back(std::move(local_plan));
plans.emplace_back(std::move(remote_plan));
auto union_step = std::make_unique<UnionStep>(std::move(input_streams));
query_plan.unitePlans(std::move(union_step), std::move(plans));
}
else
{
chassert(max_replicas_to_use <= pools_to_use.size());
pools_to_use.resize(max_replicas_to_use);
auto read_from_remote = std::make_unique<ReadFromParallelRemoteReplicasStep>(
query_ast,
new_cluster,
storage_id,
std::move(coordinator),
header,
processed_stage,
new_context,
getThrottler(new_context),
std::move(scalars),
std::move(external_tables),
getLogger("ReadFromParallelRemoteReplicasStep"),
std::move(storage_limits),
std::move(pools_to_use));
query_plan.addStep(std::move(read_from_remote));
}
}
void executeQueryWithParallelReplicas(
@ -540,7 +669,8 @@ void executeQueryWithParallelReplicas(
const QueryTreeNodePtr & query_tree,
const PlannerContextPtr & planner_context,
ContextPtr context,
std::shared_ptr<const StorageLimitsList> storage_limits)
std::shared_ptr<const StorageLimitsList> storage_limits,
QueryPlanStepPtr analyzed_read_from_merge_tree)
{
QueryTreeNodePtr modified_query_tree = query_tree->clone();
rewriteJoinToGlobalJoin(modified_query_tree, context);
@ -550,7 +680,8 @@ void executeQueryWithParallelReplicas(
= InterpreterSelectQueryAnalyzer::getSampleBlock(modified_query_tree, context, SelectQueryOptions(processed_stage).analyze());
auto modified_query_ast = queryNodeToDistributedSelectQuery(modified_query_tree);
executeQueryWithParallelReplicas(query_plan, storage_id, header, processed_stage, modified_query_ast, context, storage_limits);
executeQueryWithParallelReplicas(
query_plan, storage_id, header, processed_stage, modified_query_ast, context, storage_limits, std::move(analyzed_read_from_merge_tree));
}
void executeQueryWithParallelReplicas(
@ -646,6 +777,49 @@ void executeQueryWithParallelReplicasCustomKey(
context, query_info.query, storage_id.getDatabaseName(), storage_id.getTableName(), /*table_function_ptr=*/nullptr);
executeQueryWithParallelReplicasCustomKey(query_plan, storage_id, query_info, columns, snapshot, processed_stage, header, context);
}
bool canUseParallelReplicasOnInitiator(const ContextPtr & context)
{
if (!context->canUseParallelReplicasOnInitiator())
return false;
auto cluster = context->getClusterForParallelReplicas();
if (cluster->getShardCount() == 1)
return cluster->getShardsInfo()[0].getAllNodeCount() > 1;
/// parallel replicas with distributed table
auto scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{};
UInt64 shard_num = 0; /// shard_num is 1-based, so 0 - no shard specified
const auto it = scalars.find("_shard_num");
if (it != scalars.end())
{
const Block & block = it->second;
const auto & column = block.safeGetByPosition(0).column;
shard_num = column->getUInt(0);
}
if (shard_num > 0)
{
const auto shard_count = cluster->getShardCount();
if (shard_num > shard_count)
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Shard number is greater than shard count: shard_num={} shard_count={} cluster={}",
shard_num,
shard_count,
cluster->getName());
return cluster->getShardsInfo().at(shard_num - 1).getAllNodeCount() > 1;
}
if (cluster->getShardCount() > 1)
throw DB::Exception(
ErrorCodes::UNEXPECTED_CLUSTER,
"`cluster_for_parallel_replicas` setting refers to cluster with {} shards. Expected a cluster with one shard",
cluster->getShardCount());
return false;
}
}
}

View File

@ -35,6 +35,9 @@ using QueryTreeNodePtr = std::shared_ptr<IQueryTreeNode>;
class PlannerContext;
using PlannerContextPtr = std::shared_ptr<PlannerContext>;
class IQueryPlanStep;
using QueryPlanStepPtr = std::unique_ptr<IQueryPlanStep>;
namespace ClusterProxy
{
@ -55,6 +58,8 @@ using AdditionalShardFilterGenerator = std::function<ASTPtr(uint64_t)>;
AdditionalShardFilterGenerator
getShardFilterGeneratorForCustomKey(const Cluster & cluster, ContextPtr context, const ColumnsDescription & columns);
bool canUseParallelReplicasOnInitiator(const ContextPtr & context);
/// Execute a distributed query, creating a query plan, from which the query pipeline can be built.
/// `stream_factory` object encapsulates the logic of creating plans for a different type of query
/// (currently SELECT, DESCRIBE).
@ -81,7 +86,8 @@ void executeQueryWithParallelReplicas(
QueryProcessingStage::Enum processed_stage,
const ASTPtr & query_ast,
ContextPtr context,
std::shared_ptr<const StorageLimitsList> storage_limits);
std::shared_ptr<const StorageLimitsList> storage_limits,
QueryPlanStepPtr read_from_merge_tree = nullptr);
void executeQueryWithParallelReplicas(
QueryPlan & query_plan,
@ -98,7 +104,8 @@ void executeQueryWithParallelReplicas(
const QueryTreeNodePtr & query_tree,
const PlannerContextPtr & planner_context,
ContextPtr context,
std::shared_ptr<const StorageLimitsList> storage_limits);
std::shared_ptr<const StorageLimitsList> storage_limits,
QueryPlanStepPtr read_from_merge_tree);
void executeQueryWithParallelReplicasCustomKey(
QueryPlan & query_plan,

View File

@ -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);

View File

@ -10,6 +10,7 @@
#include <Common/SensitiveDataMasker.h>
#include <Common/Macros.h>
#include <Common/EventNotifier.h>
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Common/Stopwatch.h>
#include <Common/formatReadable.h>
#include <Common/Throttler.h>
@ -121,7 +122,6 @@
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <base/defines.h>
namespace fs = std::filesystem;
namespace ProfileEvents
@ -164,6 +164,9 @@ namespace CurrentMetrics
extern const Metric TablesLoaderForegroundThreadsActive;
extern const Metric TablesLoaderForegroundThreadsScheduled;
extern const Metric IOWriterThreadsScheduled;
extern const Metric BuildVectorSimilarityIndexThreads;
extern const Metric BuildVectorSimilarityIndexThreadsActive;
extern const Metric BuildVectorSimilarityIndexThreadsScheduled;
extern const Metric AttachedTable;
extern const Metric AttachedView;
extern const Metric AttachedDictionary;
@ -297,6 +300,8 @@ struct ContextSharedPart : boost::noncopyable
mutable std::unique_ptr<ThreadPool> load_marks_threadpool; /// Threadpool for loading marks cache.
mutable OnceFlag prefetch_threadpool_initialized;
mutable std::unique_ptr<ThreadPool> prefetch_threadpool; /// Threadpool for loading marks cache.
mutable OnceFlag build_vector_similarity_index_threadpool_initialized;
mutable std::unique_ptr<ThreadPool> build_vector_similarity_index_threadpool; /// Threadpool for vector-similarity index creation.
mutable UncompressedCachePtr index_uncompressed_cache TSA_GUARDED_BY(mutex); /// The cache of decompressed blocks for MergeTree indices.
mutable QueryCachePtr query_cache TSA_GUARDED_BY(mutex); /// Cache of query results.
mutable MarkCachePtr index_mark_cache TSA_GUARDED_BY(mutex); /// Cache of marks in compressed files of MergeTree indices.
@ -3297,6 +3302,21 @@ size_t Context::getPrefetchThreadpoolSize() const
return config.getUInt(".prefetch_threadpool_pool_size", 100);
}
ThreadPool & Context::getBuildVectorSimilarityIndexThreadPool() const
{
callOnce(shared->build_vector_similarity_index_threadpool_initialized, [&] {
size_t pool_size = shared->server_settings.max_build_vector_similarity_index_thread_pool_size > 0
? shared->server_settings.max_build_vector_similarity_index_thread_pool_size
: getNumberOfPhysicalCPUCores();
shared->build_vector_similarity_index_threadpool = std::make_unique<ThreadPool>(
CurrentMetrics::BuildVectorSimilarityIndexThreads,
CurrentMetrics::BuildVectorSimilarityIndexThreadsActive,
CurrentMetrics::BuildVectorSimilarityIndexThreadsScheduled,
pool_size);
});
return *shared->build_vector_similarity_index_threadpool;
}
BackgroundSchedulePool & Context::getBufferFlushSchedulePool() const
{
callOnce(shared->buffer_flush_schedule_pool_initialized, [&] {

View File

@ -1097,6 +1097,8 @@ public:
/// and make a prefetch by putting a read task to threadpoolReader.
size_t getPrefetchThreadpoolSize() const;
ThreadPool & getBuildVectorSimilarityIndexThreadPool() const;
/// Settings for MergeTree background tasks stored in config.xml
BackgroundTaskSchedulingSettings getBackgroundProcessingTaskSchedulingSettings() const;
BackgroundTaskSchedulingSettings getBackgroundMoveTaskSchedulingSettings() const;

View File

@ -34,6 +34,23 @@ void AddedColumns<true>::buildOutput()
{
if (join_data_avg_perkey_rows < output_by_row_list_threshold)
buildOutputFromBlocks<true>();
else if (join_data_sorted)
{
for (size_t i = 0; i < this->size(); ++i)
{
auto & col = columns[i];
for (auto row_ref_i : lazy_output.row_refs)
{
if (row_ref_i)
{
const RowRefList * row_ref_list = reinterpret_cast<const RowRefList *>(row_ref_i);
col->insertRangeFrom(*row_ref_list->block->getByPosition(right_indexes[i]).column, row_ref_list->row_num, row_ref_list->rows);
}
else
type_name[i].type->insertDefaultInto(*col);
}
}
}
else
{
for (size_t i = 0; i < this->size(); ++i)

View File

@ -66,6 +66,9 @@ public:
, join_on_keys(join_on_keys_)
, additional_filter_expression(additional_filter_expression_)
, rows_to_add(left_block.rows())
, join_data_avg_perkey_rows(join.getJoinedData()->avgPerKeyRows())
, output_by_row_list_threshold(join.getTableJoin().outputByRowListPerkeyRowsThreshold())
, join_data_sorted(join.getJoinedData()->sorted)
, is_join_get(is_join_get_)
{
size_t num_columns_to_add = block_with_columns_to_add.columns();
@ -113,8 +116,6 @@ public:
if (columns[j]->isNullable() && !saved_column->isNullable())
nullable_column_ptrs[j] = typeid_cast<ColumnNullable *>(columns[j].get());
}
join_data_avg_perkey_rows = join.getJoinedData()->avgPerKeyRows();
output_by_row_list_threshold = join.getTableJoin().outputByRowListPerkeyRowsThreshold();
}
size_t size() const { return columns.size(); }
@ -149,6 +150,7 @@ public:
bool output_by_row_list = false;
size_t join_data_avg_perkey_rows = 0;
size_t output_by_row_list_threshold = 0;
bool join_data_sorted = false;
IColumn::Filter filter;
void reserve(bool need_replicate)

View File

@ -649,7 +649,6 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
}
data->keys_to_join = total_rows;
shrinkStoredBlocksToFit(total_bytes);
return table_join->sizeLimits().check(total_rows, total_bytes, "JOIN", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
}
@ -1237,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;
@ -1361,4 +1361,96 @@ bool HashJoin::needUsedFlagsForPerRightTableRow(std::shared_ptr<TableJoin> table
return false;
}
template <JoinKind KIND, typename Map, JoinStrictness STRICTNESS>
void HashJoin::tryRerangeRightTableDataImpl(Map & map [[maybe_unused]])
{
constexpr JoinFeatures<KIND, STRICTNESS, Map> join_features;
if constexpr (!join_features.is_all_join || (!join_features.left && !join_features.inner))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Only left or inner join table can be reranged.");
else
{
auto merge_rows_into_one_block = [&](BlocksList & blocks, RowRefList & rows_ref)
{
auto it = rows_ref.begin();
if (it.ok())
{
if (blocks.empty() || blocks.back().rows() >= DEFAULT_BLOCK_SIZE)
blocks.emplace_back(it->block->cloneEmpty());
}
else
{
return;
}
auto & block = blocks.back();
size_t start_row = block.rows();
for (; it.ok(); ++it)
{
for (size_t i = 0; i < block.columns(); ++i)
{
auto & col = block.getByPosition(i).column->assumeMutableRef();
col.insertFrom(*it->block->getByPosition(i).column, it->row_num);
}
}
if (block.rows() > start_row)
{
RowRefList new_rows_ref(&block, start_row, block.rows() - start_row);
rows_ref = std::move(new_rows_ref);
}
};
auto visit_rows_map = [&](BlocksList & blocks, MapsAll & rows_map)
{
switch (data->type)
{
#define M(TYPE) \
case Type::TYPE: \
{\
rows_map.TYPE->forEachMapped([&](RowRefList & rows_ref) { merge_rows_into_one_block(blocks, rows_ref); }); \
break; \
}
APPLY_FOR_JOIN_VARIANTS(M)
#undef M
default:
break;
}
};
BlocksList sorted_blocks;
visit_rows_map(sorted_blocks, map);
data->blocks.swap(sorted_blocks);
}
}
void HashJoin::tryRerangeRightTableData()
{
if (!table_join->allowJoinSorting() || table_join->getMixedJoinExpression() || !isInnerOrLeft(kind) || strictness != JoinStrictness::All)
return;
/// We should not rerange the right table on such conditions:
/// 1. the right table is already reranged by key or it is empty.
/// 2. the join clauses size is greater than 1, like `...join on a.key1=b.key1 or a.key2=b.key2`, we can not rerange the right table on different set of keys.
/// 3. the number of right table rows exceed the threshold, which may result in a significant cost for reranging and lead to performance degradation.
/// 4. the keys of right table is very sparse, which may result in insignificant performance improvement after reranging by key.
if (!data || data->sorted || data->blocks.empty() || data->maps.size() > 1 || data->rows_to_join > table_join->sortRightMaximumTableRows() || data->avgPerKeyRows() < table_join->sortRightMinimumPerkeyRows())
return;
if (data->keys_to_join == 0)
data->keys_to_join = getTotalRowCount();
/// If the there is no columns to add, means no columns to output, then the rerange would not improve performance by using column's `insertRangeFrom`
/// to replace column's `insertFrom` to make the output.
if (sample_block_with_columns_to_add.columns() == 0)
{
LOG_DEBUG(log, "The joined right table total rows :{}, total keys :{}", data->rows_to_join, data->keys_to_join);
return;
}
[[maybe_unused]] bool result = joinDispatch(
kind,
strictness,
data->maps.front(),
/*prefer_use_maps_all*/ false,
[&](auto kind_, auto strictness_, auto & map_) { tryRerangeRightTableDataImpl<kind_, decltype(map_), strictness_>(map_); });
chassert(result);
data->sorted = true;
}
}

View File

@ -345,11 +345,12 @@ public:
size_t blocks_allocated_size = 0;
size_t blocks_nullmaps_allocated_size = 0;
/// Number of rows of right table to join
size_t rows_to_join = 0;
/// Number of keys of right table to join
size_t keys_to_join = 0;
/// Whether the right table reranged by key
bool sorted = false;
size_t avgPerKeyRows() const
{
@ -465,6 +466,10 @@ private:
void validateAdditionalFilterExpression(std::shared_ptr<ExpressionActions> additional_filter_expression);
bool needUsedFlagsForPerRightTableRow(std::shared_ptr<TableJoin> table_join_) const;
void tryRerangeRightTableData() override;
template <JoinKind KIND, typename Map, JoinStrictness STRICTNESS>
void tryRerangeRightTableDataImpl(Map & map);
};
}

View File

@ -83,7 +83,6 @@ public:
const Block & block_with_columns_to_add,
const MapsTemplateVector & maps_,
bool is_join_get = false);
private:
template <typename KeyGetter, bool is_asof_join>
static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes);
@ -199,4 +198,3 @@ extern template class HashJoinMethods<JoinKind::Full, JoinStrictness::Semi, Hash
extern template class HashJoinMethods<JoinKind::Full, JoinStrictness::Anti, HashJoin::MapsOne>;
extern template class HashJoinMethods<JoinKind::Full, JoinStrictness::Asof, HashJoin::MapsAsof>;
}

View File

@ -115,6 +115,7 @@ public:
/// Peek next stream of delayed joined blocks.
virtual IBlocksStreamPtr getDelayedBlocks() { return nullptr; }
virtual bool hasDelayedBlocks() const { return false; }
virtual void tryRerangeRightTableData() {}
virtual IBlocksStreamPtr
getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const = 0;

View File

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

View File

@ -123,6 +123,7 @@ struct RowRefList : RowRef
RowRefList() {} /// NOLINT
RowRefList(const Block * block_, size_t row_num_) : RowRef(block_, row_num_), rows(1) {}
RowRefList(const Block * block_, size_t row_start_, size_t rows_) : RowRef(block_, row_start_), rows(static_cast<SizeT>(rows_)) {}
ForwardIterator begin() const { return ForwardIterator(this); }

View File

@ -45,7 +45,7 @@ Chunk Squashing::squash(Chunk && input_chunk)
Chunk Squashing::add(Chunk && input_chunk)
{
if (!input_chunk)
if (!input_chunk || input_chunk.getNumRows() == 0)
return {};
/// Just read block is already enough.

View File

@ -116,6 +116,9 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_, Temporary
, max_files_to_merge(settings.join_on_disk_max_files_to_merge)
, temporary_files_codec(settings.temporary_files_codec)
, output_by_rowlist_perkey_rows_threshold(settings.join_output_by_rowlist_perkey_rows_threshold)
, sort_right_minimum_perkey_rows(settings.join_to_sort_minimum_perkey_rows)
, sort_right_maximum_table_rows(settings.join_to_sort_maximum_table_rows)
, allow_join_sorting(settings.allow_experimental_join_right_table_sorting)
, max_memory_usage(settings.max_memory_usage)
, tmp_volume(tmp_volume_)
, tmp_data(tmp_data_)

View File

@ -149,6 +149,9 @@ private:
const size_t max_files_to_merge = 0;
const String temporary_files_codec = "LZ4";
const size_t output_by_rowlist_perkey_rows_threshold = 0;
const size_t sort_right_minimum_perkey_rows = 0;
const size_t sort_right_maximum_table_rows = 0;
const bool allow_join_sorting = false;
/// Value if setting max_memory_usage for query, can be used when max_bytes_in_join is not specified.
size_t max_memory_usage = 0;
@ -297,6 +300,9 @@ public:
}
size_t outputByRowListPerkeyRowsThreshold() const { return output_by_rowlist_perkey_rows_threshold; }
size_t sortRightMinimumPerkeyRows() const { return sort_right_minimum_perkey_rows; }
size_t sortRightMaximumTableRows() const { return sort_right_maximum_table_rows; }
bool allowJoinSorting() const { return allow_join_sorting; }
size_t defaultMaxBytes() const { return default_max_bytes; }
size_t maxJoinedBlockRows() const { return max_joined_block_rows; }
size_t maxRowsInRightBlock() const { return partial_merge_join_rows_in_right_blocks; }

View File

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

View File

@ -99,6 +99,7 @@ namespace DB
namespace ErrorCodes
{
extern const int QUERY_CACHE_USED_WITH_NONDETERMINISTIC_FUNCTIONS;
extern const int QUERY_CACHE_USED_WITH_NON_THROW_OVERFLOW_MODE;
extern const int QUERY_CACHE_USED_WITH_SYSTEM_TABLE;
extern const int INTO_OUTFILE_NOT_ALLOWED;
extern const int INVALID_TRANSACTION;
@ -1121,6 +1122,21 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
&& (ast->as<ASTSelectQuery>() || ast->as<ASTSelectWithUnionQuery>());
QueryCache::Usage query_cache_usage = QueryCache::Usage::None;
/// Bug 67476: If the query runs with a non-THROW overflow mode and hits a limit, the query cache will store a truncated result (if
/// enabled). This is incorrect. Unfortunately it is hard to detect from the perspective of the query cache that the query result
/// is truncated. Therefore throw an exception, to notify the user to disable either the query cache or use another overflow mode.
if (settings.use_query_cache && (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))
throw Exception(ErrorCodes::QUERY_CACHE_USED_WITH_NON_THROW_OVERFLOW_MODE, "use_query_cache and overflow_mode != 'throw' cannot be used together");
/// If the query runs with "use_query_cache = 1", we first probe if the query cache already contains the query result (if yes:
/// return result from cache). If doesn't, we execute the query normally and write the result into the query cache. Both steps use a
/// hash of the AST, the current database and the settings as cache key. Unfortunately, the settings are in some places internally

View File

@ -74,7 +74,8 @@ private:
findMySQLFunctionSecretArguments();
}
else if ((function.name == "s3") || (function.name == "cosn") || (function.name == "oss") ||
(function.name == "deltaLake") || (function.name == "hudi") || (function.name == "iceberg"))
(function.name == "deltaLake") || (function.name == "hudi") || (function.name == "iceberg") ||
(function.name == "gcs"))
{
/// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...)
findS3FunctionSecretArguments(/* is_cluster_function= */ false);

View File

@ -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;
}

View File

@ -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;
}

View File

@ -892,6 +892,30 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
/// query_plan can be empty if there is nothing to read
if (query_plan.isInitialized() && parallel_replicas_enabled_for_storage(storage, settings))
{
if (query_context->canUseParallelReplicasCustomKey() && query_context->getClientInfo().distributed_depth == 0)
{
if (auto cluster = query_context->getClusterForParallelReplicas();
query_context->canUseParallelReplicasCustomKeyForCluster(*cluster))
{
planner_context->getMutableQueryContext()->setSetting("prefer_localhost_replica", Field{0});
auto modified_query_info = select_query_info;
modified_query_info.cluster = std::move(cluster);
from_stage = QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit;
QueryPlan query_plan_parallel_replicas;
ClusterProxy::executeQueryWithParallelReplicasCustomKey(
query_plan_parallel_replicas,
storage->getStorageID(),
modified_query_info,
storage->getInMemoryMetadataPtr()->getColumns(),
storage_snapshot,
from_stage,
table_expression_query_info.query_tree,
query_context);
query_plan = std::move(query_plan_parallel_replicas);
}
}
else if (ClusterProxy::canUseParallelReplicasOnInitiator(query_context))
{
// (1) find read step
QueryPlan::Node * node = query_plan.getRootNode();
@ -918,43 +942,23 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
}
chassert(reading);
if (query_context->canUseParallelReplicasCustomKey() && query_context->getClientInfo().distributed_depth == 0)
{
if (auto cluster = query_context->getClusterForParallelReplicas();
query_context->canUseParallelReplicasCustomKeyForCluster(*cluster))
{
planner_context->getMutableQueryContext()->setSetting("prefer_localhost_replica", Field{0});
auto modified_query_info = select_query_info;
modified_query_info.cluster = std::move(cluster);
from_stage = QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit;
QueryPlan query_plan_parallel_replicas;
ClusterProxy::executeQueryWithParallelReplicasCustomKey(
query_plan_parallel_replicas,
storage->getStorageID(),
modified_query_info,
storage->getInMemoryMetadataPtr()->getColumns(),
storage_snapshot,
from_stage,
table_expression_query_info.query_tree,
query_context);
query_plan = std::move(query_plan_parallel_replicas);
}
}
else if (query_context->canUseParallelReplicasOnInitiator())
{
// (2) if it's ReadFromMergeTree - run index analysis and check number of rows to read
if (settings.parallel_replicas_min_number_of_rows_per_replica > 0)
{
auto result_ptr = reading->selectRangesToRead();
UInt64 rows_to_read = result_ptr->selected_rows;
reading->setAnalyzedResult(std::move(result_ptr));
if (table_expression_query_info.trivial_limit > 0 && table_expression_query_info.trivial_limit < rows_to_read)
rows_to_read = table_expression_query_info.trivial_limit;
if (max_block_size_limited && (max_block_size_limited < rows_to_read))
rows_to_read = max_block_size_limited;
const size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica;
const size_t number_of_replicas_to_use
= rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica;
LOG_TRACE(
getLogger("Planner"),
"Estimated {} rows to read. It is enough work for {} parallel replicas",
@ -980,6 +984,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
{
from_stage = QueryProcessingStage::WithMergeableState;
QueryPlan query_plan_parallel_replicas;
QueryPlanStepPtr reading_step = std::move(node->step);
ClusterProxy::executeQueryWithParallelReplicas(
query_plan_parallel_replicas,
storage->getStorageID(),
@ -987,9 +992,24 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
table_expression_query_info.query_tree,
table_expression_query_info.planner_context,
query_context,
table_expression_query_info.storage_limits);
table_expression_query_info.storage_limits,
std::move(reading_step));
query_plan = std::move(query_plan_parallel_replicas);
}
else
{
QueryPlan query_plan_no_parallel_replicas;
storage->read(
query_plan_no_parallel_replicas,
columns_names,
storage_snapshot,
table_expression_query_info,
query_context,
from_stage,
max_block_size,
max_streams);
query_plan = std::move(query_plan_no_parallel_replicas);
}
}
}

View File

@ -52,7 +52,13 @@ std::stack<const QueryNode *> getSupportingParallelReplicasQuery(const IQueryTre
const auto & storage = table_node.getStorage();
/// Here we check StorageDummy as well, to support a query tree with replaced storages.
if (std::dynamic_pointer_cast<MergeTreeData>(storage) || typeid_cast<const StorageDummy *>(storage.get()))
{
/// parallel replicas is not supported with FINAL
if (table_node.getTableExpressionModifiers() && table_node.getTableExpressionModifiers()->hasFinal())
return {};
return res;
}
return {};
}

View File

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

View File

@ -0,0 +1,32 @@
#include <Processors/QueryPlan/QueryPlan.h>
#include <Interpreters/ActionsDAG.h>
#include <Processors/QueryPlan/ExpressionStep.h>
namespace DB
{
void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects)
{
if (blocksHaveEqualStructure(plan.getCurrentDataStream().header, header))
return;
auto mode = has_missing_objects ? ActionsDAG::MatchColumnsMode::Position : ActionsDAG::MatchColumnsMode::Name;
auto get_converting_dag = [mode](const Block & block_, const Block & header_)
{
/// Convert header structure to expected.
/// Also we ignore constants from result and replace it with constants from header.
/// It is needed for functions like `now64()` or `randConstant()` because their values may be different.
return ActionsDAG::makeConvertingActions(
block_.getColumnsWithTypeAndName(),
header_.getColumnsWithTypeAndName(),
mode,
true);
};
auto convert_actions_dag = get_converting_dag(plan.getCurrentDataStream().header, header);
auto converting = std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), std::move(convert_actions_dag));
plan.addStep(std::move(converting));
}
}

View File

@ -0,0 +1,9 @@
#pragma once
namespace DB
{
class QueryPlan;
class Block;
void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects);
}

View File

@ -2,43 +2,13 @@
#include <Common/checkStackSize.h>
#include <Core/Settings.h>
#include <Interpreters/ActionsDAG.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/ConvertingActions.h>
namespace DB
{
namespace
{
void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects)
{
if (blocksHaveEqualStructure(plan.getCurrentDataStream().header, header))
return;
auto mode = has_missing_objects ? ActionsDAG::MatchColumnsMode::Position : ActionsDAG::MatchColumnsMode::Name;
auto get_converting_dag = [mode](const Block & block_, const Block & header_)
{
/// Convert header structure to expected.
/// Also we ignore constants from result and replace it with constants from header.
/// It is needed for functions like `now64()` or `randConstant()` because their values may be different.
return ActionsDAG::makeConvertingActions(
block_.getColumnsWithTypeAndName(),
header_.getColumnsWithTypeAndName(),
mode,
true);
};
auto convert_actions_dag = get_converting_dag(plan.getCurrentDataStream().header, header);
auto converting = std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), std::move(convert_actions_dag));
plan.addStep(std::move(converting));
}
}
std::unique_ptr<QueryPlan> createLocalPlan(
const ASTPtr & query_ast,
const Block & header,

View File

@ -18,6 +18,7 @@
#include <Processors/QueryPlan/Optimizations/Optimizations.h>
#include <Processors/QueryPlan/Optimizations/actionsDAGUtils.h>
#include <Processors/QueryPlan/ReadFromMergeTree.h>
#include <Processors/QueryPlan/ReadFromRemote.h>
#include <Processors/QueryPlan/SortingStep.h>
#include <Processors/QueryPlan/TotalsHavingStep.h>
#include <Processors/QueryPlan/UnionStep.h>
@ -899,6 +900,18 @@ AggregationInputOrder buildInputOrderInfo(AggregatingStep & aggregating, QueryPl
return {};
}
static bool readingFromParallelReplicas(const QueryPlan::Node * node)
{
IQueryPlanStep * step = node->step.get();
while (!node->children.empty())
{
step = node->children.front()->step.get();
node = node->children.front();
}
return typeid_cast<const ReadFromParallelRemoteReplicasStep *>(step);
}
void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes)
{
if (node.children.size() != 1)
@ -924,6 +937,16 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes)
std::vector<InputOrderInfoPtr> infos;
infos.reserve(node.children.size());
for (const auto * child : union_node->children)
{
/// in case of parallel replicas
/// avoid applying read-in-order optimization for local replica
/// since it will lead to different parallel replicas modes
/// between local and remote nodes
if (readingFromParallelReplicas(child))
return;
}
for (auto * child : union_node->children)
{
infos.push_back(buildInputOrderInfo(*sorting, *child, steps_to_update));

View File

@ -0,0 +1,97 @@
#include <Processors/QueryPlan/ParallelReplicasLocalPlan.h>
#include <Common/checkStackSize.h>
#include <Interpreters/ActionsDAG.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
#include <Interpreters/StorageID.h>
#include <Parsers/ASTFunction.h>
#include <Processors/QueryPlan/ConvertingActions.h>
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/ISourceStep.h>
#include <Processors/QueryPlan/ReadFromMergeTree.h>
#include <Processors/Sources/NullSource.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <Processors/Transforms/FilterTransform.h>
#include <QueryPipeline/Pipe.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
#include <Storages/MergeTree/RequestResponse.h>
namespace DB
{
std::pair<std::unique_ptr<QueryPlan>, bool> createLocalPlanForParallelReplicas(
const ASTPtr & query_ast,
const Block & header,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
ParallelReplicasReadingCoordinatorPtr coordinator,
QueryPlanStepPtr analyzed_read_from_merge_tree,
size_t replica_number)
{
checkStackSize();
auto query_plan = std::make_unique<QueryPlan>();
auto new_context = Context::createCopy(context);
/// Do not push down limit to local plan, as it will break `rows_before_limit_at_least` counter.
if (processed_stage == QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit)
processed_stage = QueryProcessingStage::WithMergeableStateAfterAggregation;
/// Do not apply AST optimizations, because query
/// is already optimized and some optimizations
/// can be applied only for non-distributed tables
/// and we can produce query, inconsistent with remote plans.
auto select_query_options = SelectQueryOptions(processed_stage).ignoreASTOptimizations();
/// For Analyzer, identifier in GROUP BY/ORDER BY/LIMIT BY lists has been resolved to
/// ConstantNode in QueryTree if it is an alias of a constant, so we should not replace
/// ConstantNode with ProjectionNode again(https://github.com/ClickHouse/ClickHouse/issues/62289).
new_context->setSetting("enable_positional_arguments", Field(false));
new_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0));
auto interpreter = InterpreterSelectQueryAnalyzer(query_ast, new_context, select_query_options);
query_plan = std::make_unique<QueryPlan>(std::move(interpreter).extractQueryPlan());
QueryPlan::Node * node = query_plan->getRootNode();
ReadFromMergeTree * reading = nullptr;
while (node)
{
reading = typeid_cast<ReadFromMergeTree *>(node->step.get());
if (reading)
break;
if (!node->children.empty())
node = node->children.at(0);
else
node = nullptr;
}
if (!reading)
/// it can happened if merge tree table is empty, - it'll be replaced with ReadFromPreparedSource
return {std::move(query_plan), false};
ReadFromMergeTree::AnalysisResultPtr analyzed_result_ptr;
if (analyzed_read_from_merge_tree.get())
{
auto * analyzed_merge_tree = typeid_cast<ReadFromMergeTree *>(analyzed_read_from_merge_tree.get());
if (analyzed_merge_tree)
analyzed_result_ptr = analyzed_merge_tree->getAnalyzedResult();
}
MergeTreeAllRangesCallback all_ranges_cb = [coordinator](InitialAllRangesAnnouncement announcement)
{ coordinator->handleInitialAllRangesAnnouncement(std::move(announcement)); };
MergeTreeReadTaskCallback read_task_cb = [coordinator](ParallelReadRequest req) -> std::optional<ParallelReadResponse>
{ return coordinator->handleRequest(std::move(req)); };
auto read_from_merge_tree_parallel_replicas = reading->createLocalParallelReplicasReadingStep(
analyzed_result_ptr, std::move(all_ranges_cb), std::move(read_task_cb), replica_number);
node->step = std::move(read_from_merge_tree_parallel_replicas);
addConvertingActions(*query_plan, header, /*has_missing_objects=*/false);
return {std::move(query_plan), true};
}
}

View File

@ -0,0 +1,19 @@
#pragma once
#include <Core/QueryProcessingStage.h>
#include <Parsers/IAST_fwd.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h>
namespace DB
{
std::pair<std::unique_ptr<QueryPlan>, bool> createLocalPlanForParallelReplicas(
const ASTPtr & query_ast,
const Block & header,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
ParallelReplicasReadingCoordinatorPtr coordinator,
QueryPlanStepPtr read_from_merge_tree,
size_t replica_number);
}

View File

@ -277,7 +277,10 @@ ReadFromMergeTree::ReadFromMergeTree(
std::shared_ptr<PartitionIdToMaxBlock> max_block_numbers_to_read_,
LoggerPtr log_,
AnalysisResultPtr analyzed_result_ptr_,
bool enable_parallel_reading)
bool enable_parallel_reading_,
std::optional<MergeTreeAllRangesCallback> all_ranges_callback_,
std::optional<MergeTreeReadTaskCallback> read_task_callback_,
std::optional<size_t> number_of_current_replica_)
: SourceStepWithFilter(DataStream{.header = MergeTreeSelectProcessor::transformHeader(
storage_snapshot_->getSampleBlockForColumns(all_column_names_),
query_info_.prewhere_info)}, all_column_names_, query_info_, storage_snapshot_, context_)
@ -295,12 +298,20 @@ ReadFromMergeTree::ReadFromMergeTree(
, max_block_numbers_to_read(std::move(max_block_numbers_to_read_))
, log(std::move(log_))
, analyzed_result_ptr(analyzed_result_ptr_)
, is_parallel_reading_from_replicas(enable_parallel_reading)
, is_parallel_reading_from_replicas(enable_parallel_reading_)
, enable_remove_parts_from_snapshot_optimization(query_info_.merge_tree_enable_remove_parts_from_snapshot_optimization)
, number_of_current_replica(number_of_current_replica_)
{
if (is_parallel_reading_from_replicas)
{
if (all_ranges_callback_.has_value())
all_ranges_callback = all_ranges_callback_.value();
else
all_ranges_callback = context->getMergeTreeAllRangesCallback();
if (read_task_callback_.has_value())
read_task_callback = read_task_callback_.value();
else
read_task_callback = context->getMergeTreeReadTaskCallback();
}
@ -335,11 +346,33 @@ ReadFromMergeTree::ReadFromMergeTree(
enable_vertical_final);
}
std::unique_ptr<ReadFromMergeTree> ReadFromMergeTree::createLocalParallelReplicasReadingStep(
AnalysisResultPtr analyzed_result_ptr_,
MergeTreeAllRangesCallback all_ranges_callback_,
MergeTreeReadTaskCallback read_task_callback_,
size_t replica_number)
{
const bool enable_parallel_reading = true;
return std::make_unique<ReadFromMergeTree>(
prepared_parts,
mutations_snapshot,
all_column_names,
data,
getQueryInfo(),
getStorageSnapshot(),
getContext(),
block_size.max_block_size_rows,
requested_num_streams,
max_block_numbers_to_read,
log,
std::move(analyzed_result_ptr_),
enable_parallel_reading,
all_ranges_callback_,
read_task_callback_,
replica_number);
}
Pipe ReadFromMergeTree::readFromPoolParallelReplicas(
RangesInDataParts parts_with_range,
Names required_columns,
PoolSettings pool_settings)
Pipe ReadFromMergeTree::readFromPoolParallelReplicas(RangesInDataParts parts_with_range, Names required_columns, PoolSettings pool_settings)
{
const auto & client_info = context->getClientInfo();
@ -347,7 +380,7 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas(
{
.all_callback = all_ranges_callback.value(),
.callback = read_task_callback.value(),
.number_of_current_replica = client_info.number_of_current_replica,
.number_of_current_replica = number_of_current_replica.value_or(client_info.number_of_current_replica),
};
/// We have a special logic for local replica. It has to read less data, because in some cases it should
@ -529,7 +562,7 @@ Pipe ReadFromMergeTree::readInOrder(
{
.all_callback = all_ranges_callback.value(),
.callback = read_task_callback.value(),
.number_of_current_replica = client_info.number_of_current_replica,
.number_of_current_replica = number_of_current_replica.value_or(client_info.number_of_current_replica),
};
auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier;
@ -584,11 +617,12 @@ Pipe ReadFromMergeTree::readInOrder(
context);
}
/// Actually it means that parallel reading from replicas enabled
/// and we have to collaborate with initiator.
/// In this case we won't set approximate rows, because it will be accounted multiple times.
const auto in_order_limit = query_info.input_order_info ? query_info.input_order_info->limit : 0;
const bool set_total_rows_approx = !is_parallel_reading_from_replicas;
/// If parallel replicas enabled, set total rows in progress here only on initiator with local plan
/// Otherwise rows will counted multiple times
const UInt64 in_order_limit = query_info.input_order_info ? query_info.input_order_info->limit : 0;
const bool parallel_replicas_local_plan_for_initiator = is_parallel_reading_from_replicas
&& context->getSettingsRef().parallel_replicas_local_plan && context->canUseParallelReplicasOnInitiator();
const bool set_total_rows_approx = !is_parallel_reading_from_replicas || parallel_replicas_local_plan_for_initiator;
Pipes pipes;
for (size_t i = 0; i < parts_with_ranges.size(); ++i)
@ -1422,11 +1456,8 @@ static void buildIndexes(
const auto & settings = context->getSettingsRef();
indexes.emplace(ReadFromMergeTree::Indexes{{
filter_actions_dag,
context,
primary_key_column_names,
primary_key.expression}, {}, {}, {}, {}, false, {}});
indexes.emplace(
ReadFromMergeTree::Indexes{KeyCondition{filter_actions_dag, context, primary_key_column_names, primary_key.expression}});
if (metadata_snapshot->hasPartitionKey())
{

View File

@ -121,7 +121,16 @@ public:
std::shared_ptr<PartitionIdToMaxBlock> max_block_numbers_to_read_,
LoggerPtr log_,
AnalysisResultPtr analyzed_result_ptr_,
bool enable_parallel_reading);
bool enable_parallel_reading_,
std::optional<MergeTreeAllRangesCallback> all_ranges_callback_ = std::nullopt,
std::optional<MergeTreeReadTaskCallback> read_task_callback_ = std::nullopt,
std::optional<size_t> number_of_current_replica_ = std::nullopt);
std::unique_ptr<ReadFromMergeTree> createLocalParallelReplicasReadingStep(
AnalysisResultPtr analyzed_result_ptr_,
MergeTreeAllRangesCallback all_ranges_callback_,
MergeTreeReadTaskCallback read_task_callback_,
size_t replica_number);
static constexpr auto name = "ReadFromMergeTree";
String getName() const override { return name; }
@ -143,6 +152,11 @@ public:
struct Indexes
{
explicit Indexes(KeyCondition key_condition_)
: key_condition(std::move(key_condition_))
, use_skip_indexes(false)
{}
KeyCondition key_condition;
std::optional<PartitionPruner> partition_pruner;
std::optional<KeyCondition> minmax_idx_condition;
@ -268,6 +282,7 @@ private:
std::optional<MergeTreeReadTaskCallback> read_task_callback;
bool enable_vertical_final = false;
bool enable_remove_parts_from_snapshot_optimization = true;
std::optional<size_t> number_of_current_replica;
};
}

View File

@ -21,8 +21,8 @@
#include <Client/ConnectionPoolWithFailover.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Parsers/ASTFunction.h>
#include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h>
#include <boost/algorithm/string/join.hpp>
#include <fmt/format.h>
namespace DB
{
@ -362,6 +362,7 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep(
ASTPtr query_ast_,
ClusterPtr cluster_,
const StorageID & storage_id_,
ParallelReplicasReadingCoordinatorPtr coordinator_,
Block header_,
QueryProcessingStage::Enum stage_,
ContextMutablePtr context_,
@ -369,11 +370,14 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep(
Scalars scalars_,
Tables external_tables_,
LoggerPtr log_,
std::shared_ptr<const StorageLimitsList> storage_limits_)
std::shared_ptr<const StorageLimitsList> storage_limits_,
std::vector<ConnectionPoolPtr> pools_to_use_,
std::optional<size_t> exclude_pool_index_)
: ISourceStep(DataStream{.header = std::move(header_)})
, cluster(cluster_)
, query_ast(query_ast_)
, storage_id(storage_id_)
, coordinator(std::move(coordinator_))
, stage(std::move(stage_))
, context(context_)
, throttler(throttler_)
@ -381,16 +385,24 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep(
, external_tables{external_tables_}
, storage_limits(std::move(storage_limits_))
, log(log_)
, pools_to_use(std::move(pools_to_use_))
, exclude_pool_index(exclude_pool_index_)
{
chassert(cluster->getShardCount() == 1);
std::vector<String> description;
description.push_back(fmt::format("query: {}", formattedAST(query_ast)));
std::vector<String> replicas;
replicas.reserve(pools_to_use.size());
for (const auto & pool : cluster->getShardsInfo().front().per_replica_pools)
description.push_back(fmt::format("Replica: {}", pool->getHost()));
for (size_t i = 0, l = pools_to_use.size(); i < l; ++i)
{
if (exclude_pool_index.has_value() && i == exclude_pool_index)
continue;
setStepDescription(boost::algorithm::join(description, ", "));
replicas.push_back(pools_to_use[i]->getAddress());
}
auto description = fmt::format("Query: {} Replicas: {}", formattedAST(query_ast), fmt::join(replicas, ", "));
setStepDescription(std::move(description));
}
void ReadFromParallelRemoteReplicasStep::enforceSorting(SortDescription output_sort_description)
@ -406,48 +418,29 @@ void ReadFromParallelRemoteReplicasStep::enforceAggregationInOrder()
void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
{
Pipes pipes;
const Settings & current_settings = context->getSettingsRef();
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings);
const auto & shard = cluster->getShardsInfo().at(0);
size_t max_replicas_to_use = current_settings.max_parallel_replicas;
if (max_replicas_to_use > shard.getAllNodeCount())
std::vector<std::string_view> addresses;
addresses.reserve(pools_to_use.size());
for (size_t i = 0, l = pools_to_use.size(); i < l; ++i)
{
LOG_INFO(
getLogger("ReadFromParallelRemoteReplicasStep"),
"The number of replicas requested ({}) is bigger than the real number available in the cluster ({}). "
"Will use the latter number to execute the query.",
current_settings.max_parallel_replicas,
shard.getAllNodeCount());
max_replicas_to_use = shard.getAllNodeCount();
if (exclude_pool_index.has_value() && i == exclude_pool_index)
continue;
addresses.emplace_back(pools_to_use[i]->getAddress());
}
LOG_DEBUG(getLogger("ReadFromParallelRemoteReplicasStep"), "Addresses to use: {}", fmt::join(addresses, ", "));
std::vector<ConnectionPoolWithFailover::Base::ShuffledPool> shuffled_pool;
if (max_replicas_to_use < shard.getAllNodeCount())
for (size_t i = 0, l = pools_to_use.size(); i < l; ++i)
{
shuffled_pool = shard.pool->getShuffledPools(current_settings);
shuffled_pool.resize(max_replicas_to_use);
}
else
{
/// try to preserve replicas order if all replicas in cluster are used for query execution
/// it's important for data locality during query execution
auto priority_func = [](size_t i) { return Priority{static_cast<Int64>(i)}; };
shuffled_pool = shard.pool->getShuffledPools(current_settings, priority_func);
}
if (exclude_pool_index.has_value() && i == exclude_pool_index)
continue;
coordinator
= std::make_shared<ParallelReplicasReadingCoordinator>(max_replicas_to_use, current_settings.parallel_replicas_mark_segment_size);
for (size_t i=0; i < max_replicas_to_use; ++i)
{
IConnections::ReplicaInfo replica_info
{
IConnections::ReplicaInfo replica_info{
/// we should use this number specifically because efficiency of data distribution by consistent hash depends on it.
.number_of_current_replica = i,
};
addPipeForSingeReplica(pipes, shuffled_pool[i].pool, replica_info);
addPipeForSingeReplica(pipes, pools_to_use[i], replica_info);
}
auto pipe = Pipe::unitePipes(std::move(pipes));

View File

@ -70,6 +70,7 @@ public:
ASTPtr query_ast_,
ClusterPtr cluster_,
const StorageID & storage_id_,
ParallelReplicasReadingCoordinatorPtr coordinator_,
Block header_,
QueryProcessingStage::Enum stage_,
ContextMutablePtr context_,
@ -77,7 +78,9 @@ public:
Scalars scalars_,
Tables external_tables_,
LoggerPtr log_,
std::shared_ptr<const StorageLimitsList> storage_limits_);
std::shared_ptr<const StorageLimitsList> storage_limits_,
std::vector<ConnectionPoolPtr> pools_to_use,
std::optional<size_t> exclude_pool_index_ = std::nullopt);
String getName() const override { return "ReadFromRemoteParallelReplicas"; }
@ -100,6 +103,8 @@ private:
Tables external_tables;
std::shared_ptr<const StorageLimitsList> storage_limits;
LoggerPtr log;
std::vector<ConnectionPoolPtr> pools_to_use;
std::optional<size_t> exclude_pool_index;
};
}

View File

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

View File

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

View File

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

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