Merge branch 'master' into refactor-read-metrics-and-callbacks

This commit is contained in:
Nikolai Kochetov 2022-05-26 11:07:45 +00:00
commit 84f97b53de
126 changed files with 3661 additions and 2101 deletions

View File

@ -25,9 +25,11 @@ The following versions of ClickHouse server are currently being supported with s
| 21.10 | :x: |
| 21.11 | :x: |
| 21.12 | :x: |
| 22.1 | |
| 22.2 | |
| 22.1 | :x: |
| 22.2 | :x: |
| 22.3 | ✅ |
| 22.4 | ✅ |
| 22.5 | ✅ |
## Reporting a Vulnerability

2
contrib/cctz vendored

@ -1 +1 @@
Subproject commit 9edd0861d8328b2ae77e8fb5f4d7dcd1cf33b42b
Subproject commit 8c71d74bdf76c3fa401da845089ae60a6c0aeefa

View File

@ -63,7 +63,7 @@ RUN arch=${TARGETARCH:-amd64} \
&& chown clickhouse:clickhouse /var/lib/clickhouse \
&& chown root:clickhouse /var/log/clickhouse-server \
&& chmod +x /entrypoint.sh \
&& apk add --no-cache su-exec bash tzdata \
&& apk add --no-cache bash tzdata \
&& cp /usr/share/zoneinfo/UTC /etc/localtime \
&& echo "UTC" > /etc/timezone \
&& chmod ugo+Xrw -R /var/lib/clickhouse /var/log/clickhouse-server /etc/clickhouse-server /etc/clickhouse-client

View File

@ -3,8 +3,6 @@ FROM ubuntu:20.04
# see https://github.com/moby/moby/issues/4032#issuecomment-192327844
ARG DEBIAN_FRONTEND=noninteractive
COPY su-exec.c /su-exec.c
# ARG for quick switch to a given ubuntu mirror
ARG apt_archive="http://archive.ubuntu.com"
RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list \
@ -19,17 +17,11 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list
locales \
wget \
tzdata \
&& apt-get install -y --no-install-recommends tcc libc-dev && \
tcc /su-exec.c -o /bin/su-exec && \
chown root:root /bin/su-exec && \
chmod 0755 /bin/su-exec && \
rm /su-exec.c && \
apt-get purge -y --auto-remove tcc libc-dev libc-dev-bin libc6-dev linux-libc-dev \
&& apt-get clean
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="deb https://packages.clickhouse.com/deb ${REPO_CHANNEL} main"
ARG VERSION=22.1.1.*
ARG VERSION=22.5.1.*
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
# set non-empty deb_location_url url to create a docker image
@ -51,21 +43,6 @@ ARG single_binary_location_url=""
# installed to prevent picking those uid / gid by some unrelated software.
# The same uid / gid (101) is used both for alpine and ubuntu.
# To drop privileges, we need 'su' command, that simply changes uid and gid.
# In fact, the 'su' command from Linux is not so simple, due to inherent vulnerability in Linux:
# https://ruderich.org/simon/notes/su-sudo-from-root-tty-hijacking
# It has to mitigate this drawback of Linux, and to do this, 'su' command is creating it's own pseudo-terminal
# and forwarding commands. Due to some ridiculous curcumstances, it does not work in Docker (or it does)
# and for these reasons people are using alternatives to the 'su' command in Docker,
# that don't mess with the terminal, don't care about closing the opened files, etc...
# but can only be safe to drop privileges inside Docker.
# The question - what implementation of 'su' command to use.
# It should be a simple script doing about just two syscalls.
# Some people tend to use 'gosu' tool that is written in Go.
# It is not used for several reasons:
# 1. Dependency on some foreign code in yet another programming language - does not sound alright.
# 2. Anselmo D. Adams suggested not to use it due to false positive alarms in some undisclosed security scanners.
ARG TARGETARCH
RUN arch=${TARGETARCH:-amd64} \

View File

@ -15,29 +15,15 @@ CLICKHOUSE_GID="${CLICKHOUSE_GID:-"$(id -g clickhouse)"}"
if [ "$(id -u)" = "0" ]; then
USER=$CLICKHOUSE_UID
GROUP=$CLICKHOUSE_GID
if command -v gosu &> /dev/null; then
gosu="gosu $USER:$GROUP"
elif command -v su-exec &> /dev/null; then
gosu="su-exec $USER:$GROUP"
else
echo "No gosu/su-exec detected!"
exit 1
fi
else
USER="$(id -u)"
GROUP="$(id -g)"
gosu=""
DO_CHOWN=0
fi
# set some vars
CLICKHOUSE_CONFIG="${CLICKHOUSE_CONFIG:-/etc/clickhouse-server/config.xml}"
if ! $gosu test -f "$CLICKHOUSE_CONFIG" -a -r "$CLICKHOUSE_CONFIG"; then
echo "Configuration file '$CLICKHOUSE_CONFIG' isn't readable by user with id '$USER'"
exit 1
fi
# get CH directories locations
DATA_DIR="$(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key=path || true)"
TMP_DIR="$(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key=tmp_path || true)"
@ -65,12 +51,7 @@ do
# check if variable not empty
[ -z "$dir" ] && continue
# ensure directories exist
if [ "$DO_CHOWN" = "1" ]; then
mkdir="mkdir"
else
mkdir="$gosu mkdir"
fi
if ! $mkdir -p "$dir"; then
if ! mkdir -p "$dir"; then
echo "Couldn't create necessary directory: $dir"
exit 1
fi
@ -81,9 +62,6 @@ do
if [ "$(stat -c %u "$dir")" != "$USER" ] || [ "$(stat -c %g "$dir")" != "$GROUP" ]; then
chown -R "$USER:$GROUP" "$dir"
fi
elif ! $gosu test -d "$dir" -a -w "$dir" -a -r "$dir"; then
echo "Necessary directory '$dir' isn't accessible by user with id '$USER'"
exit 1
fi
done
@ -117,7 +95,7 @@ if [ -n "$(ls /docker-entrypoint-initdb.d/)" ] || [ -n "$CLICKHOUSE_DB" ]; then
HTTP_PORT="$(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key=http_port)"
# Listen only on localhost until the initialization is done
$gosu /usr/bin/clickhouse-server --config-file="$CLICKHOUSE_CONFIG" -- --listen_host=127.0.0.1 &
/usr/bin/clickhouse su "${USER}:${GROUP}" /usr/bin/clickhouse-server --config-file="$CLICKHOUSE_CONFIG" -- --listen_host=127.0.0.1 &
pid="$!"
# check if clickhouse is ready to accept connections
@ -173,7 +151,7 @@ if [[ $# -lt 1 ]] || [[ "$1" == "--"* ]]; then
# so the container can't be finished by ctrl+c
CLICKHOUSE_WATCHDOG_ENABLE=${CLICKHOUSE_WATCHDOG_ENABLE:-0}
export CLICKHOUSE_WATCHDOG_ENABLE
exec $gosu /usr/bin/clickhouse-server --config-file="$CLICKHOUSE_CONFIG" "$@"
/usr/bin/clickhouse su "${USER}:${GROUP}" /usr/bin/clickhouse-server --config-file="$CLICKHOUSE_CONFIG" "$@"
fi
# Otherwise, we assume the user want to run his own process, for example a `bash` shell to explore this image

View File

@ -1,138 +0,0 @@
/*
https://github.com/ncopa/su-exec
The file is copy-pasted verbatim to avoid supply chain attacks.
The MIT License (MIT)
Copyright (c) 2015 ncopa
Permission is hereby granted, free of charge, to any person obtaining a copy
of this software and associated documentation files (the "Software"), to deal
in the Software without restriction, including without limitation the rights
to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
copies of the Software, and to permit persons to whom the Software is
furnished to do so, subject to the following conditions:
The above copyright notice and this permission notice shall be included in all
copies or substantial portions of the Software.
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
SOFTWARE.
*/
/* set user and group id and exec */
#include <sys/types.h>
#include <err.h>
#include <errno.h>
#include <grp.h>
#include <pwd.h>
#include <stdio.h>
#include <stdlib.h>
#include <string.h>
#include <unistd.h>
static char *argv0;
static void usage(int exitcode)
{
printf("Usage: %s user-spec command [args]\n", argv0);
exit(exitcode);
}
int main(int argc, char *argv[])
{
char *user, *group, **cmdargv;
char *end;
uid_t uid = getuid();
gid_t gid = getgid();
argv0 = argv[0];
if (argc < 3)
usage(0);
user = argv[1];
group = strchr(user, ':');
if (group)
*group++ = '\0';
cmdargv = &argv[2];
struct passwd *pw = NULL;
if (user[0] != '\0') {
uid_t nuid = strtol(user, &end, 10);
if (*end == '\0')
uid = nuid;
else {
pw = getpwnam(user);
if (pw == NULL)
err(1, "getpwnam(%s)", user);
}
}
if (pw == NULL) {
pw = getpwuid(uid);
}
if (pw != NULL) {
uid = pw->pw_uid;
gid = pw->pw_gid;
}
setenv("HOME", pw != NULL ? pw->pw_dir : "/", 1);
if (group && group[0] != '\0') {
/* group was specified, ignore grouplist for setgroups later */
pw = NULL;
gid_t ngid = strtol(group, &end, 10);
if (*end == '\0')
gid = ngid;
else {
struct group *gr = getgrnam(group);
if (gr == NULL)
err(1, "getgrnam(%s)", group);
gid = gr->gr_gid;
}
}
if (pw == NULL) {
if (setgroups(1, &gid) < 0)
err(1, "setgroups(%i)", gid);
} else {
int ngroups = 0;
gid_t *glist = NULL;
while (1) {
int r = getgrouplist(pw->pw_name, gid, glist, &ngroups);
if (r >= 0) {
if (setgroups(ngroups, glist) < 0)
err(1, "setgroups");
break;
}
glist = realloc(glist, ngroups * sizeof(gid_t));
if (glist == NULL)
err(1, "malloc");
}
}
if (setgid(gid) < 0)
err(1, "setgid(%i)", gid);
if (setuid(uid) < 0)
err(1, "setuid(%i)", uid);
execvp(cmdargv[0], cmdargv);
err(1, "%s", cmdargv[0]);
return 1;
}

View File

@ -121,7 +121,7 @@ There are ordinary functions and aggregate functions. For aggregate functions, s
Ordinary functions do not change the number of rows they work as if they are processing each row independently. In fact, functions are not called for individual rows, but for `Block`s of data to implement vectorized query execution.
There are some miscellaneous functions, like [blockSize](../sql-reference/functions/other-functions.md#function-blocksize), [rowNumberInBlock](../sql-reference/functions/other-functions.md#function-rownumberinblock), and [runningAccumulate](../sql-reference/functions/other-functions.md#runningaccumulate), that exploit block processing and violate the independence of rows.
There are some miscellaneous functions, like [blockSize](../sql-reference/functions/other-functions.md#blocksize-function-blocksize), [rowNumberInBlock](../sql-reference/functions/other-functions.md#rownumberinblock-function-rownumberinblock), and [runningAccumulate](../sql-reference/functions/other-functions.md#runningaccumulate-runningaccumulate), that exploit block processing and violate the independence of rows.
ClickHouse has strong typing, so theres no implicit type conversion. If a function does not support a specific combination of types, it throws an exception. But functions can work (be overloaded) for many different combinations of types. For example, the `plus` function (to implement the `+` operator) works for any combination of numeric types: `UInt8` + `Float32`, `UInt16` + `Int8`, and so on. Also, some variadic functions can accept any number of arguments, such as the `concat` function.

View File

@ -94,14 +94,11 @@ SELECT library_name, license_type, license_path FROM system.licenses ORDER BY li
[Example](https://gh-api.clickhouse.com/play?user=play#U0VMRUNUIGxpYnJhcnlfbmFtZSwgbGljZW5zZV90eXBlLCBsaWNlbnNlX3BhdGggRlJPTSBzeXN0ZW0ubGljZW5zZXMgT1JERVIgQlkgbGlicmFyeV9uYW1lIENPTExBVEUgJ2VuJw==)
## Guidelines for adding new third-party libraries and maintaining custom changes in them {#adding-third-party-libraries}
## Adding new third-party libraries and maintaining patches in third-party libraries {#adding-third-party-libraries}
1. All external third-party code should reside in the dedicated directories under `contrib` directory of ClickHouse repo. Prefer Git submodules, when available.
2. Fork/mirror the official repo in [Clickhouse-extras](https://github.com/ClickHouse-Extras). Prefer official GitHub repos, when available.
3. Branch from the branch you want to integrate, e.g., `master` -> `clickhouse/master`, or `release/vX.Y.Z` -> `clickhouse/release/vX.Y.Z`.
4. All forks in [Clickhouse-extras](https://github.com/ClickHouse-Extras) can be automatically synchronized with upstreams. `clickhouse/...` branches will remain unaffected, since virtually nobody is going to use that naming pattern in their upstream repos.
5. Add submodules under `contrib` of ClickHouse repo that refer the above forks/mirrors. Set the submodules to track the corresponding `clickhouse/...` branches.
6. Every time the custom changes have to be made in the library code, a dedicated branch should be created, like `clickhouse/my-fix`. Then this branch should be merged into the branch, that is tracked by the submodule, e.g., `clickhouse/master` or `clickhouse/release/vX.Y.Z`.
7. No code should be pushed in any branch of the forks in [Clickhouse-extras](https://github.com/ClickHouse-Extras), whose names do not follow `clickhouse/...` pattern.
8. Always write the custom changes with the official repo in mind. Once the PR is merged from (a feature/fix branch in) your personal fork into the fork in [Clickhouse-extras](https://github.com/ClickHouse-Extras), and the submodule is bumped in ClickHouse repo, consider opening another PR from (a feature/fix branch in) the fork in [Clickhouse-extras](https://github.com/ClickHouse-Extras) to the official repo of the library. This will make sure, that 1) the contribution has more than a single use case and importance, 2) others will also benefit from it, 3) the change will not remain a maintenance burden solely on ClickHouse developers.
9. When a submodule needs to start using a newer code from the original branch (e.g., `master`), and since the custom changes might be merged in the branch it is tracking (e.g., `clickhouse/master`) and so it may diverge from its original counterpart (i.e., `master`), a careful merge should be carried out first, i.e., `master` -> `clickhouse/master`, and only then the submodule can be bumped in ClickHouse.
1. Each third-party libary must reside in a dedicated directory under the `contrib/` directory of the ClickHouse repository. Avoid dumps/copies of external code, instead use Git's submodule feature to pull third-party code from an external upstream repository.
2. Submodules are listed in `.gitmodule`. If the external library can be used as-is, you may reference the upstream repository directly. Otherwise, i.e. the external libary requires patching/customization, create a fork of the official repository in the [Clickhouse organization in GitHub](https://github.com/ClickHouse).
3. In the latter case, create a branch with `clickhouse/` prefix from the branch you want to integrate, e.g. `clickhouse/master` (for `master`) or `clickhouse/release/vX.Y.Z` (for a `release/vX.Y.Z` tag). The purpose of this branch is to isolate customization of the library from upstream work. For example, pulls from the upstream repository into the fork will leave all `clickhouse/` branches unaffected. Submodules in `contrib/` must only track `clickhouse/` branches of forked third-party repositories.
4. To patch a fork of a third-party library, create a dedicated branch with `clickhouse/` prefix in the fork, e.g. `clickhouse/fix-some-desaster`. Finally, merge the patch branch into the custom tracking branch (e.g. `clickhouse/master` or `clickhouse/release/vX.Y.Z`) using a PR.
5. Always create patches of third-party libraries with the official repository in mind. Once a PR of a patch branch to the `clickhouse/` branch in the fork repository is done and the submodule version in ClickHouse's official repository is bumped, consider opening another PR from the patch branch to the upstream library repository. This ensures, that 1) the contribution has more than a single use case and importance, 2) others will also benefit from it, 3) the change will not remain a maintenance burden solely on ClickHouse developers.
9. To update a submodule with changes in the upstream repository, first merge upstream `master` (or a new `versionX.Y.Z` tag) into the `clickhouse`-tracking branch in the fork repository. Conflicts with patches/customization will need to be resolved in this merge (see Step 4.). Once the merge is done, bump the submodule in ClickHouse to point to the new hash in the fork.

View File

@ -5,15 +5,15 @@ sidebar_label: ClickHouse Keeper
# ClickHouse Keeper {#clickHouse-keeper}
ClickHouse server uses [ZooKeeper](https://zookeeper.apache.org/) coordination system for data [replication](../engines/table-engines/mergetree-family/replication.md) and [distributed DDL](../sql-reference/distributed-ddl.md) queries execution. ClickHouse Keeper is an alternative coordination system compatible with ZooKeeper.
ClickHouse Keeper provides the coordination system for data [replication](../engines/table-engines/mergetree-family/replication.md) and [distributed DDL](../sql-reference/distributed-ddl.md) queries execution. ClickHouse Keeper is compatible with ZooKeeper.
## Implementation details {#implementation-details}
ZooKeeper is one of the first well-known open-source coordination systems. It's implemented in Java, has quite a simple and powerful data model. ZooKeeper's coordination algorithm called ZAB (ZooKeeper Atomic Broadcast) doesn't provide linearizability guarantees for reads, because each ZooKeeper node serves reads locally. Unlike ZooKeeper ClickHouse Keeper is written in C++ and uses [RAFT algorithm](https://raft.github.io/) [implementation](https://github.com/eBay/NuRaft). This algorithm allows to have linearizability for reads and writes, has several open-source implementations in different languages.
ZooKeeper is one of the first well-known open-source coordination systems. It's implemented in Java, and has quite a simple and powerful data model. ZooKeeper's coordination algorithm, ZooKeeper Atomic Broadcast (ZAB), doesn't provide linearizability guarantees for reads, because each ZooKeeper node serves reads locally. Unlike ZooKeeper ClickHouse Keeper is written in C++ and uses the [RAFT algorithm](https://raft.github.io/) [implementation](https://github.com/eBay/NuRaft). This algorithm allows linearizability for reads and writes, and has several open-source implementations in different languages.
By default, ClickHouse Keeper provides the same guarantees as ZooKeeper (linearizable writes, non-linearizable reads). It has a compatible client-server protocol, so any standard ZooKeeper client can be used to interact with ClickHouse Keeper. Snapshots and logs have an incompatible format with ZooKeeper, but `clickhouse-keeper-converter` tool allows to convert ZooKeeper data to ClickHouse Keeper snapshot. Interserver protocol in ClickHouse Keeper is also incompatible with ZooKeeper so mixed ZooKeeper / ClickHouse Keeper cluster is impossible.
By default, ClickHouse Keeper provides the same guarantees as ZooKeeper (linearizable writes, non-linearizable reads). It has a compatible client-server protocol, so any standard ZooKeeper client can be used to interact with ClickHouse Keeper. Snapshots and logs have an incompatible format with ZooKeeper, but the `clickhouse-keeper-converter` tool enables the conversion of ZooKeeper data to ClickHouse Keeper snapshots. The interserver protocol in ClickHouse Keeper is also incompatible with ZooKeeper so a mixed ZooKeeper / ClickHouse Keeper cluster is impossible.
ClickHouse Keeper supports Access Control List (ACL) the same way as [ZooKeeper](https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html#sc_ZooKeeperAccessControl) does. ClickHouse Keeper supports the same set of permissions and has the identical built-in schemes: `world`, `auth`, `digest`, `host` and `ip`. Digest authentication scheme uses pair `username:password`. Password is encoded in Base64.
ClickHouse Keeper supports Access Control Lists (ACLs) the same way as [ZooKeeper](https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html#sc_ZooKeeperAccessControl) does. ClickHouse Keeper supports the same set of permissions and has the identical built-in schemes: `world`, `auth` and `digest`. The digest authentication scheme uses the pair `username:password`, the password is encoded in Base64.
:::note
External integrations are not supported.
@ -21,25 +21,25 @@ External integrations are not supported.
## Configuration {#configuration}
ClickHouse Keeper can be used as a standalone replacement for ZooKeeper or as an internal part of the ClickHouse server, but in both cases configuration is almost the same `.xml` file. The main ClickHouse Keeper configuration tag is `<keeper_server>`. Keeper configuration has the following parameters:
ClickHouse Keeper can be used as a standalone replacement for ZooKeeper or as an internal part of the ClickHouse server. In both cases the configuration is almost the same `.xml` file. The main ClickHouse Keeper configuration tag is `<keeper_server>`. Keeper configuration has the following parameters:
- `tcp_port` — Port for a client to connect (default for ZooKeeper is `2181`).
- `tcp_port_secure` — Secure port for an SSL connection between client and keeper-server.
- `server_id` — Unique server id, each participant of the ClickHouse Keeper cluster must have a unique number (1, 2, 3, and so on).
- `log_storage_path` — Path to coordination logs, better to store logs on the non-busy device (same for ZooKeeper).
- `log_storage_path` — Path to coordination logs, just like ZooKeeper it is best to store logs on non-busy nodes.
- `snapshot_storage_path` — Path to coordination snapshots.
Other common parameters are inherited from the ClickHouse server config (`listen_host`, `logger`, and so on).
Internal coordination settings are located in `<keeper_server>.<coordination_settings>` section:
Internal coordination settings are located in the `<keeper_server>.<coordination_settings>` section:
- `operation_timeout_ms` — Timeout for a single client operation (ms) (default: 10000).
- `min_session_timeout_ms` — Min timeout for client session (ms) (default: 10000).
- `session_timeout_ms` — Max timeout for client session (ms) (default: 100000).
- `dead_session_check_period_ms` — How often ClickHouse Keeper check dead sessions and remove them (ms) (default: 500).
- `dead_session_check_period_ms` — How often ClickHouse Keeper checks for dead sessions and removes them (ms) (default: 500).
- `heart_beat_interval_ms` — How often a ClickHouse Keeper leader will send heartbeats to followers (ms) (default: 500).
- `election_timeout_lower_bound_ms` — If the follower didn't receive heartbeats from the leader in this interval, then it can initiate leader election (default: 1000).
- `election_timeout_upper_bound_ms` — If the follower didn't receive heartbeats from the leader in this interval, then it must initiate leader election (default: 2000).
- `election_timeout_lower_bound_ms` — If the follower does not receive a heartbeat from the leader in this interval, then it can initiate leader election (default: 1000).
- `election_timeout_upper_bound_ms` — If the follower does not receive a heartbeat from the leader in this interval, then it must initiate leader election (default: 2000).
- `rotate_log_storage_interval` — How many log records to store in a single file (default: 100000).
- `reserved_log_items` — How many coordination log records to store before compaction (default: 100000).
- `snapshot_distance` — How often ClickHouse Keeper will create new snapshots (in the number of records in logs) (default: 100000).
@ -55,7 +55,7 @@ Internal coordination settings are located in `<keeper_server>.<coordination_set
- `startup_timeout` — If the server doesn't connect to other quorum participants in the specified timeout it will terminate (ms) (default: 30000).
- `four_letter_word_white_list` — White list of 4lw commands (default: "conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro").
Quorum configuration is located in `<keeper_server>.<raft_configuration>` section and contain servers description.
Quorum configuration is located in the `<keeper_server>.<raft_configuration>` section and contain servers description.
The only parameter for the whole quorum is `secure`, which enables encrypted connection for communication between quorum participants. The parameter can be set `true` if SSL connection is required for internal communication between nodes, or left unspecified otherwise.
@ -66,7 +66,7 @@ The main parameters for each `<server>` are:
- `port` — Port where this server listens for connections.
:::note
In the case of a change in the topology of your ClickHouse Keeper cluster (eg. replacing a server), please make sure to keep the mapping `server_id` to `hostname` consistent and avoid shuffling or reusing an existing `server_id` for different servers (eg. it can happen if your rely on automation scripts to deploy ClickHouse Keeper)
In the case of a change in the topology of your ClickHouse Keeper cluster (e.g., replacing a server), please make sure to keep the mapping of `server_id` to `hostname` consistent and avoid shuffling or reusing an existing `server_id` for different servers (e.g., it can happen if your rely on automation scripts to deploy ClickHouse Keeper)
:::
Examples of configuration for quorum with three nodes can be found in [integration tests](https://github.com/ClickHouse/ClickHouse/tree/master/tests/integration) with `test_keeper_` prefix. Example configuration for server #1:
@ -112,7 +112,7 @@ ClickHouse Keeper is bundled into the ClickHouse server package, just add config
clickhouse-keeper --config /etc/your_path_to_config/config.xml
```
If you don't have the symlink (`clickhouse-keeper`) you can create it or specify `keeper` as argument:
If you don't have the symlink (`clickhouse-keeper`) you can create it or specify `keeper` as an argument to `clickhouse`:
```bash
clickhouse keeper --config /etc/your_path_to_config/config.xml

View File

@ -34,7 +34,7 @@ You can also declare attributes as coming from environment variables by using `f
The config can also define “substitutions”. If an element has the `incl` attribute, the corresponding substitution from the file will be used as the value. By default, the path to the file with substitutions is `/etc/metrika.xml`. This can be changed in the [include_from](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-include_from) element in the server config. The substitution values are specified in `/clickhouse/substitution_name` elements in this file. If a substitution specified in `incl` does not exist, it is recorded in the log. To prevent ClickHouse from logging missing substitutions, specify the `optional="true"` attribute (for example, settings for [macros](../operations/server-configuration-parameters/settings.md#macros)).
If you want to replace an entire element with a substitution use `include` as element name.
If you want to replace an entire element with a substitution use `include` as the element name.
XML substitution example:

View File

@ -114,7 +114,7 @@ A large number of parts in a table reduces performance of ClickHouse queries and
## replicated_deduplication_window {#replicated-deduplication-window}
The number of most recently inserted blocks for which Zookeeper stores hash sums to check for duplicates.
The number of most recently inserted blocks for which ClickHouse Keeper stores hash sums to check for duplicates.
Possible values:
@ -123,7 +123,7 @@ Possible values:
Default value: 100.
The `Insert` command creates one or more blocks (parts). When inserting into Replicated tables, ClickHouse for [insert deduplication](../../engines/table-engines/mergetree-family/replication/) writes the hash sums of the created parts into Zookeeper. Hash sums are stored only for the most recent `replicated_deduplication_window` blocks. The oldest hash sums are removed from Zookeeper.
The `Insert` command creates one or more blocks (parts). For [insert deduplication](../../engines/table-engines/mergetree-family/replication/), when writing into replicated tables, ClickHouse writes the hash sums of the created parts into ClickHouse Keeper. Hash sums are stored only for the most recent `replicated_deduplication_window` blocks. The oldest hash sums are removed from ClickHouse Keeper.
A large number of `replicated_deduplication_window` slows down `Inserts` because it needs to compare more entries.
The hash sum is calculated from the composition of the field names and types and the data of the inserted part (stream of bytes).
@ -142,7 +142,7 @@ A deduplication mechanism is used, similar to replicated tables (see [replicated
## replicated_deduplication_window_seconds {#replicated-deduplication-window-seconds}
The number of seconds after which the hash sums of the inserted blocks are removed from Zookeeper.
The number of seconds after which the hash sums of the inserted blocks are removed from ClickHouse Keeper.
Possible values:
@ -150,7 +150,7 @@ Possible values:
Default value: 604800 (1 week).
Similar to [replicated_deduplication_window](#replicated-deduplication-window), `replicated_deduplication_window_seconds` specifies how long to store hash sums of blocks for insert deduplication. Hash sums older than `replicated_deduplication_window_seconds` are removed from Zookeeper, even if they are less than ` replicated_deduplication_window`.
Similar to [replicated_deduplication_window](#replicated-deduplication-window), `replicated_deduplication_window_seconds` specifies how long to store hash sums of blocks for insert deduplication. Hash sums older than `replicated_deduplication_window_seconds` are removed from ClickHouse Keeper, even if they are less than ` replicated_deduplication_window`.
## replicated_fetches_http_connection_timeout {#replicated_fetches_http_connection_timeout}

View File

@ -1838,7 +1838,7 @@ Usage
By default, deduplication is not performed for materialized views but is done upstream, in the source table.
If an INSERTed block is skipped due to deduplication in the source table, there will be no insertion into attached materialized views. This behaviour exists to enable the insertion of highly aggregated data into materialized views, for cases where inserted blocks are the same after materialized view aggregation but derived from different INSERTs into the source table.
At the same time, this behaviour “breaks” `INSERT` idempotency. If an `INSERT` into the main table was successful and `INSERT` into a materialized view failed (e.g. because of communication failure with Zookeeper) a client will get an error and can retry the operation. However, the materialized view wont receive the second insert because it will be discarded by deduplication in the main (source) table. The setting `deduplicate_blocks_in_dependent_materialized_views` allows for changing this behaviour. On retry, a materialized view will receive the repeat insert and will perform a deduplication check by itself,
At the same time, this behaviour “breaks” `INSERT` idempotency. If an `INSERT` into the main table was successful and `INSERT` into a materialized view failed (e.g. because of communication failure with ClickHouse Keeper) a client will get an error and can retry the operation. However, the materialized view wont receive the second insert because it will be discarded by deduplication in the main (source) table. The setting `deduplicate_blocks_in_dependent_materialized_views` allows for changing this behaviour. On retry, a materialized view will receive the repeat insert and will perform a deduplication check by itself,
ignoring check result for the source table, and will insert rows lost because of the first failure.
## insert_deduplication_token {#insert_deduplication_token}
@ -2459,7 +2459,7 @@ Default value: 0.
## merge_selecting_sleep_ms {#merge_selecting_sleep_ms}
Sleep time for merge selecting when no part is selected. A lower setting triggers selecting tasks in `background_schedule_pool` frequently, which results in a large number of requests to Zookeeper in large-scale clusters.
Sleep time for merge selecting when no part is selected. A lower setting triggers selecting tasks in `background_schedule_pool` frequently, which results in a large number of requests to ClickHouse Keeper in large-scale clusters.
Possible values:
@ -2607,7 +2607,7 @@ Default value: 128.
## background_fetches_pool_size {#background_fetches_pool_size}
Sets the number of threads performing background fetches for [replicated](../../engines/table-engines/mergetree-family/replication.md) tables. This setting is applied at the ClickHouse server start and cant be changed in a user session. For production usage with frequent small insertions or slow ZooKeeper cluster is recommended to use default value.
Sets the number of threads performing background fetches for [replicated](../../engines/table-engines/mergetree-family/replication.md) tables. This setting is applied at the ClickHouse server start and cant be changed in a user session. For production usage with frequent small insertions or slow ZooKeeper cluster it is recommended to use default value.
Possible values:

View File

@ -15,7 +15,7 @@ Columns:
- `query_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Query start time.
- `query_finish_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Query finish time.
- `query_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Duration of query execution (in milliseconds).
- `exception_code` ([Enum8](../../sql-reference/data-types/enum.md)) — Exception code from [ZooKeeper](../../operations/tips.md#zookeeper).
- `exception_code` ([Enum8](../../sql-reference/data-types/enum.md)) — Exception code from [ClickHouse Keeper](../../operations/tips.md#zookeeper).
**Example**

View File

@ -8,7 +8,7 @@ Columns:
- `table` ([String](../../sql-reference/data-types/string.md)) — The name of the table to which the mutation was applied.
- `mutation_id` ([String](../../sql-reference/data-types/string.md)) — The ID of the mutation. For replicated tables these IDs correspond to znode names in the `<table_path_in_zookeeper>/mutations/` directory in ZooKeeper. For non-replicated tables the IDs correspond to file names in the data directory of the table.
- `mutation_id` ([String](../../sql-reference/data-types/string.md)) — The ID of the mutation. For replicated tables these IDs correspond to znode names in the `<table_path_in_clickhouse_keeper>/mutations/` directory in ClickHouse Keeper. For non-replicated tables the IDs correspond to file names in the data directory of the table.
- `command` ([String](../../sql-reference/data-types/string.md)) — The mutation command string (the part of the query after `ALTER TABLE [db.]table`).

View File

@ -62,13 +62,13 @@ Columns:
Note that writes can be performed to any replica that is available and has a session in ZK, regardless of whether it is a leader.
- `can_become_leader` (`UInt8`) - Whether the replica can be a leader.
- `is_readonly` (`UInt8`) - Whether the replica is in read-only mode.
This mode is turned on if the config does not have sections with ZooKeeper, if an unknown error occurred when reinitializing sessions in ZooKeeper, and during session reinitialization in ZooKeeper.
- `is_session_expired` (`UInt8`) - the session with ZooKeeper has expired. Basically the same as `is_readonly`.
This mode is turned on if the config does not have sections with ClickHouse Keeper, if an unknown error occurred when reinitializing sessions in ClickHouse Keeper, and during session reinitialization in ClickHouse Keeper.
- `is_session_expired` (`UInt8`) - the session with ClickHouse Keeper has expired. Basically the same as `is_readonly`.
- `future_parts` (`UInt32`) - The number of data parts that will appear as the result of INSERTs or merges that havent been done yet.
- `parts_to_check` (`UInt32`) - The number of data parts in the queue for verification. A part is put in the verification queue if there is suspicion that it might be damaged.
- `zookeeper_path` (`String`) - Path to table data in ZooKeeper.
- `replica_name` (`String`) - Replica name in ZooKeeper. Different replicas of the same table have different names.
- `replica_path` (`String`) - Path to replica data in ZooKeeper. The same as concatenating zookeeper_path/replicas/replica_path.
- `zookeeper_path` (`String`) - Path to table data in ClickHouse Keeper.
- `replica_name` (`String`) - Replica name in ClickHouse Keeper. Different replicas of the same table have different names.
- `replica_path` (`String`) - Path to replica data in ClickHouse Keeper. The same as concatenating zookeeper_path/replicas/replica_path.
- `columns_version` (`Int32`) - Version number of the table structure. Indicates how many times ALTER was performed. If replicas have different versions, it means some replicas havent made all of the ALTERs yet.
- `queue_size` (`UInt32`) - Size of the queue for operations waiting to be performed. Operations include inserting blocks of data, merges, and certain other actions. It usually coincides with `future_parts`.
- `inserts_in_queue` (`UInt32`) - Number of inserts of blocks of data that need to be made. Insertions are usually replicated fairly quickly. If this number is large, it means something is wrong.
@ -86,12 +86,12 @@ The next 4 columns have a non-zero value only where there is an active session w
- `last_queue_update` (`DateTime`) - When the queue was updated last time.
- `absolute_delay` (`UInt64`) - How big lag in seconds the current replica has.
- `total_replicas` (`UInt8`) - The total number of known replicas of this table.
- `active_replicas` (`UInt8`) - The number of replicas of this table that have a session in ZooKeeper (i.e., the number of functioning replicas).
- `active_replicas` (`UInt8`) - The number of replicas of this table that have a session in ClickHouse Keeper (i.e., the number of functioning replicas).
- `last_queue_update_exception` (`String`) - When the queue contains broken entries. Especially important when ClickHouse breaks backward compatibility between versions and log entries written by newer versions aren't parseable by old versions.
- `zookeeper_exception` (`String`) - The last exception message, got if the error happened when fetching the info from ZooKeeper.
- `zookeeper_exception` (`String`) - The last exception message, got if the error happened when fetching the info from ClickHouse Keeper.
- `replica_is_active` ([Map(String, UInt8)](../../sql-reference/data-types/map.md)) — Map between replica name and is replica active.
If you request all the columns, the table may work a bit slowly, since several reads from ZooKeeper are made for each row.
If you request all the columns, the table may work a bit slowly, since several reads from ClickHouse Keeper are made for each row.
If you do not request the last 4 columns (log_max_index, log_pointer, total_replicas, active_replicas), the table works quickly.
For example, you can check that everything is working correctly like this:

View File

@ -1,6 +1,6 @@
# replication_queue {#system_tables-replication_queue}
Contains information about tasks from replication queues stored in ZooKeeper for tables in the `ReplicatedMergeTree` family.
Contains information about tasks from replication queues stored in Clickhouse Keeper, or ZooKeeper, for tables in the `ReplicatedMergeTree` family.
Columns:
@ -8,11 +8,11 @@ Columns:
- `table` ([String](../../sql-reference/data-types/string.md)) — Name of the table.
- `replica_name` ([String](../../sql-reference/data-types/string.md)) — Replica name in ZooKeeper. Different replicas of the same table have different names.
- `replica_name` ([String](../../sql-reference/data-types/string.md)) — Replica name in ClickHouse Keeper. Different replicas of the same table have different names.
- `position` ([UInt32](../../sql-reference/data-types/int-uint.md)) — Position of the task in the queue.
- `node_name` ([String](../../sql-reference/data-types/string.md)) — Node name in ZooKeeper.
- `node_name` ([String](../../sql-reference/data-types/string.md)) — Node name in ClickHouse Keeper.
- `type` ([String](../../sql-reference/data-types/string.md)) — Type of the task in the queue, one of:

View File

@ -1,7 +1,7 @@
# zookeeper {#system-zookeeper}
The table does not exist if ZooKeeper is not configured. Allows reading data from the ZooKeeper cluster defined in the config.
The query must either have a path = condition or a `path IN` condition set with the `WHERE` clause as shown below. This corresponds to the path of the children in ZooKeeper that you want to get data for.
The table does not exist unless ClickHouse Keeper or ZooKeeper is configured. The `system.zookeeper` table exposes data from the Keeper cluster defined in the config.
The query must either have a path = condition or a `path IN` condition set with the `WHERE` clause as shown below. This corresponds to the path of the children that you want to get data for.
The query `SELECT * FROM system.zookeeper WHERE path = '/clickhouse'` outputs data for all children on the `/clickhouse` node.
To output data for all root nodes, write path = /.
@ -9,7 +9,7 @@ If the path specified in path does not exist, an exception will be thrown.
The query `SELECT * FROM system.zookeeper WHERE path IN ('/', '/clickhouse')` outputs data for all children on the `/` and `/clickhouse` node.
If in the specified path collection has does not exist path, an exception will be thrown.
It can be used to do a batch of ZooKeeper path queries.
It can be used to do a batch of Keeper path queries.
Columns:

View File

@ -118,17 +118,17 @@ in XML configuration.
This is important for ClickHouse to be able to get correct information with `cpuid` instruction.
Otherwise you may get `Illegal instruction` crashes when hypervisor is run on old CPU models.
## ZooKeeper {#zookeeper}
## ClickHouse Keeper and ZooKeeper {#zookeeper}
You are probably already using ZooKeeper for other purposes. You can use the same installation of ZooKeeper, if it isnt already overloaded.
ClickHouse Keeper is recommended to replace ZooKeeper for ClickHouse clusters. See the documentation for [ClickHouse Keeper](clickhouse-keeper.md)
Its best to use a fresh version of ZooKeeper 3.4.9 or later. The version in stable Linux distributions may be outdated.
If you would like to continue using ZooKeeper then it is best to use a fresh version of ZooKeeper 3.4.9 or later. The version in stable Linux distributions may be outdated.
You should never use manually written scripts to transfer data between different ZooKeeper clusters, because the result will be incorrect for sequential nodes. Never use the “zkcopy” utility for the same reason: https://github.com/ksprojects/zkcopy/issues/15
If you want to divide an existing ZooKeeper cluster into two, the correct way is to increase the number of its replicas and then reconfigure it as two independent clusters.
Do not run ZooKeeper on the same servers as ClickHouse. Because ZooKeeper is very sensitive for latency and ClickHouse may utilize all available system resources.
You can run ClickHouse Keeper on the same server as ClickHouse, but do not run ZooKeeper on the same servers as ClickHouse. Because ZooKeeper is very sensitive for latency and ClickHouse may utilize all available system resources.
You can have ZooKeeper observers in an ensemble but ClickHouse servers should not interact with observers.

View File

@ -11,11 +11,11 @@ Copies data from the tables in one cluster to tables in another (or the same) cl
To get a consistent copy, the data in the source tables and partitions should not change during the entire process.
:::
You can run multiple `clickhouse-copier` instances on different servers to perform the same job. ZooKeeper is used for syncing the processes.
You can run multiple `clickhouse-copier` instances on different servers to perform the same job. ClickHouse Keeper, or ZooKeeper, is used for syncing the processes.
After starting, `clickhouse-copier`:
- Connects to ZooKeeper and receives:
- Connects to ClickHouse Keeper and receives:
- Copying jobs.
- The state of the copying jobs.
@ -24,7 +24,7 @@ After starting, `clickhouse-copier`:
Each running process chooses the “closest” shard of the source cluster and copies the data into the destination cluster, resharding the data if necessary.
`clickhouse-copier` tracks the changes in ZooKeeper and applies them on the fly.
`clickhouse-copier` tracks the changes in ClickHouse Keeper and applies them on the fly.
To reduce network traffic, we recommend running `clickhouse-copier` on the same server where the source data is located.
@ -33,19 +33,19 @@ To reduce network traffic, we recommend running `clickhouse-copier` on the same
The utility should be run manually:
``` bash
$ clickhouse-copier --daemon --config zookeeper.xml --task-path /task/path --base-dir /path/to/dir
$ clickhouse-copier --daemon --config keeper.xml --task-path /task/path --base-dir /path/to/dir
```
Parameters:
- `daemon` — Starts `clickhouse-copier` in daemon mode.
- `config` — The path to the `zookeeper.xml` file with the parameters for the connection to ZooKeeper.
- `task-path` — The path to the ZooKeeper node. This node is used for syncing `clickhouse-copier` processes and storing tasks. Tasks are stored in `$task-path/description`.
- `task-file` — Optional path to file with task configuration for initial upload to ZooKeeper.
- `config` — The path to the `keeper.xml` file with the parameters for the connection to ClickHouse Keeper.
- `task-path` — The path to the ClickHouse Keeper node. This node is used for syncing `clickhouse-copier` processes and storing tasks. Tasks are stored in `$task-path/description`.
- `task-file` — Optional path to file with task configuration for initial upload to ClickHouse Keeper.
- `task-upload-force` — Force upload `task-file` even if node already exists.
- `base-dir` — The path to logs and auxiliary files. When it starts, `clickhouse-copier` creates `clickhouse-copier_YYYYMMHHSS_<PID>` subdirectories in `$base-dir`. If this parameter is omitted, the directories are created in the directory where `clickhouse-copier` was launched.
## Format of Zookeeper.xml {#format-of-zookeeper-xml}
## Format of keeper.xml {#format-of-zookeeper-xml}
``` xml
<clickhouse>

View File

@ -1141,4 +1141,261 @@ Result:
│ [590080815153545215,590080471556161535,590080677714591743,590077585338138623,590077447899185151,590079509483487231] │
└─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘
```
[Original article](https://clickhouse.com/docs/en/sql-reference/functions/geo/h3) <!--hide-->
## h3GetUnidirectionalEdge {#h3getunidirectionaledge}
Returns a unidirectional edge H3 index based on the provided origin and destination and returns 0 on error.
**Syntax**
``` sql
h3GetUnidirectionalEdge(originIndex, destinationIndex)
```
**Parameter**
- `originIndex` — Origin Hexagon index number. Type: [UInt64](../../../sql-reference/data-types/int-uint.md).
- `destinationIndex` — Destination Hexagon index number. Type: [UInt64](../../../sql-reference/data-types/int-uint.md).
**Returned value**
- Unidirectional Edge Hexagon Index number.
Type: [UInt64](../../../sql-reference/data-types/int-uint.md).
**Example**
Query:
``` sql
SELECT h3GetUnidirectionalEdge(599686042433355775, 599686043507097599) as edge;
```
Result:
``` text
┌────────────────edge─┐
│ 1248204388774707199 │
└─────────────────────┘
```
## h3UnidirectionalEdgeIsValid {#h3unidirectionaledgeisvalid}
Determines if the provided H3Index is a valid unidirectional edge index. Returns 1 if it's a unidirectional edge and 0 otherwise.
**Syntax**
``` sql
h3UnidirectionalEdgeisValid(index)
```
**Parameter**
- `index` — Hexagon index number. Type: [UInt64](../../../sql-reference/data-types/int-uint.md).
**Returned value**
- 1 — The H3 index is a valid unidirectional edge.
- 0 — The H3 index is not a valid unidirectional edge.
Type: [UInt8](../../../sql-reference/data-types/int-uint.md).
**Example**
Query:
``` sql
SELECT h3UnidirectionalEdgeIsValid(1248204388774707199) as validOrNot;
```
Result:
``` text
┌─validOrNot─┐
│ 1 │
└────────────┘
```
## h3GetOriginIndexFromUnidirectionalEdge {#h3getoriginindexfromunidirectionaledge}
Returns the origin hexagon index from the unidirectional edge H3Index.
**Syntax**
``` sql
h3GetOriginIndexFromUnidirectionalEdge(edge)
```
**Parameter**
- `edge` — Hexagon index number that represents a unidirectional edge. Type: [UInt64](../../../sql-reference/data-types/int-uint.md).
**Returned value**
- Origin Hexagon Index number.
Type: [UInt64](../../../sql-reference/data-types/int-uint.md).
**Example**
Query:
``` sql
SELECT h3GetOriginIndexFromUnidirectionalEdge(1248204388774707197) as origin;
```
Result:
``` text
┌─────────────origin─┐
│ 599686042433355773 │
└────────────────────┘
```
## h3GetDestinationIndexFromUnidirectionalEdge {#h3getdestinationindexfromunidirectionaledge}
Returns the destination hexagon index from the unidirectional edge H3Index.
**Syntax**
``` sql
h3GetDestinationIndexFromUnidirectionalEdge(edge)
```
**Parameter**
- `edge` — Hexagon index number that represents a unidirectional edge. Type: [UInt64](../../../sql-reference/data-types/int-uint.md).
**Returned value**
- Destination Hexagon Index number.
Type: [UInt64](../../../sql-reference/data-types/int-uint.md).
**Example**
Query:
``` sql
SELECT h3GetDestinationIndexFromUnidirectionalEdge(1248204388774707197) as destination;
```
Result:
``` text
┌────────destination─┐
│ 599686043507097597 │
└────────────────────┘
```
## h3GetIndexesFromUnidirectionalEdge {#h3getindexesfromunidirectionaledge}
Returns the origin and destination hexagon indexes from the given unidirectional edge H3Index.
**Syntax**
``` sql
h3GetIndexesFromUnidirectionalEdge(edge)
```
**Parameter**
- `edge` — Hexagon index number that represents a unidirectional edge. Type: [UInt64](../../../sql-reference/data-types/int-uint.md).
**Returned value**
A tuple consisting of two values `tuple(origin,destination)`:
- `origin` — Origin Hexagon index number. [UInt64](../../../sql-reference/data-types/int-uint.md).
- `destination` — Destination Hexagon index number. [UInt64](../../../sql-reference/data-types/int-uint.md).
Returns `(0,0)` if the provided input is not valid.
**Example**
Query:
``` sql
SELECT h3GetIndexesFromUnidirectionalEdge(1248204388774707199) as indexes;
```
Result:
``` text
┌─indexes─────────────────────────────────┐
│ (599686042433355775,599686043507097599) │
└─────────────────────────────────────────┘
```
## h3GetUnidirectionalEdgesFromHexagon {#h3getunidirectionaledgesfromhexagon}
Provides all of the unidirectional edges from the provided H3Index.
**Syntax**
``` sql
h3GetUnidirectionalEdgesFromHexagon(index)
```
**Parameter**
- `index` — Hexagon index number that represents a unidirectional edge. Type: [UInt64](../../../sql-reference/data-types/int-uint.md).
**Returned value**
Array of h3 indexes representing each unidirectional edge:
Type: [Array](../../../sql-reference/data-types/array.md)([UInt64](../../../sql-reference/data-types/int-uint.md)).
**Example**
Query:
``` sql
SELECT h3GetUnidirectionalEdgesFromHexagon(1248204388774707199) as edges;
```
Result:
``` text
┌─edges─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐
│ [1248204388774707199,1320261982812635135,1392319576850563071,1464377170888491007,1536434764926418943,1608492358964346879] │
└───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘
```
## h3GetUnidirectionalEdgeBoundary {#h3getunidirectionaledgeboundary}
Returns the coordinates defining the unidirectional edge.
**Syntax**
``` sql
h3GetUnidirectionalEdgeBoundary(index)
```
**Parameter**
- `index` — Hexagon index number that represents a unidirectional edge. Type: [UInt64](../../../sql-reference/data-types/int-uint.md).
**Returned value**
- Array of pairs '(lon, lat)'.
Type: [Array](../../../sql-reference/data-types/array.md)([Float64](../../../sql-reference/data-types/float.md), [Float64](../../../sql-reference/data-types/float.md)).
**Example**
Query:
``` sql
SELECT h3GetUnidirectionalEdgeBoundary(1248204388774707199) as boundary;
```
Result:
``` text
┌─boundary────────────────────────────────────────────────────────────────────────┐
│ [(37.42012867767779,-122.03773496427027),(37.33755608435299,-122.090428929044)] │
└─────────────────────────────────────────────────────────────────────────────────┘
```
[Original article](https://clickhouse.com/docs/en/sql-reference/functions/geo/h3) <!--hide-->

View File

@ -62,6 +62,8 @@ option (ENABLE_CLICKHOUSE_KEEPER "ClickHouse alternative to ZooKeeper" ${ENABLE_
option (ENABLE_CLICKHOUSE_KEEPER_CONVERTER "Util allows to convert ZooKeeper logs and snapshots into clickhouse-keeper snapshot" ${ENABLE_CLICKHOUSE_ALL})
option (ENABLE_CLICKHOUSE_SU "A tool similar to 'su'" ${ENABLE_CLICKHOUSE_ALL})
if (NOT ENABLE_NURAFT)
# RECONFIGURE_MESSAGE_LEVEL should not be used here,
# since ENABLE_NURAFT is set to OFF for FreeBSD and Darwin.
@ -237,6 +239,7 @@ add_subdirectory (install)
add_subdirectory (git-import)
add_subdirectory (bash-completion)
add_subdirectory (static-files-disk-uploader)
add_subdirectory (su)
if (ENABLE_CLICKHOUSE_KEEPER)
add_subdirectory (keeper)
@ -269,7 +272,8 @@ if (CLICKHOUSE_ONE_SHARED)
${CLICKHOUSE_ODBC_BRIDGE_SOURCES}
${CLICKHOUSE_KEEPER_SOURCES}
${CLICKHOUSE_KEEPER_CONVERTER_SOURCES}
${CLICKHOUSE_STATIC_FILES_DISK_UPLOADER_SOURCES})
${CLICKHOUSE_STATIC_FILES_DISK_UPLOADER_SOURCES}
${CLICKHOUSE_SU_SOURCES})
target_link_libraries(clickhouse-lib
${CLICKHOUSE_SERVER_LINK}
@ -285,7 +289,8 @@ if (CLICKHOUSE_ONE_SHARED)
${CLICKHOUSE_ODBC_BRIDGE_LINK}
${CLICKHOUSE_KEEPER_LINK}
${CLICKHOUSE_KEEPER_CONVERTER_LINK}
${CLICKHOUSE_STATIC_FILES_DISK_UPLOADER_LINK})
${CLICKHOUSE_STATIC_FILES_DISK_UPLOADER_LINK}
${CLICKHOUSE_SU_LINK})
target_include_directories(clickhouse-lib
${CLICKHOUSE_SERVER_INCLUDE}
@ -318,8 +323,7 @@ if (CLICKHOUSE_SPLIT_BINARY)
clickhouse-obfuscator
clickhouse-git-import
clickhouse-copier
clickhouse-static-files-disk-uploader
)
clickhouse-static-files-disk-uploader)
if (ENABLE_CLICKHOUSE_ODBC_BRIDGE)
list (APPEND CLICKHOUSE_ALL_TARGETS clickhouse-odbc-bridge)
@ -387,6 +391,9 @@ else ()
if (ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER)
clickhouse_target_link_split_lib(clickhouse static-files-disk-uploader)
endif ()
if (ENABLE_CLICKHOUSE_SU)
clickhouse_target_link_split_lib(clickhouse su)
endif ()
if (ENABLE_CLICKHOUSE_KEEPER)
clickhouse_target_link_split_lib(clickhouse keeper)
endif()
@ -453,6 +460,11 @@ else ()
install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-static-files-disk-uploader" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
list(APPEND CLICKHOUSE_BUNDLE clickhouse-static-files-disk-uploader)
endif ()
if (ENABLE_CLICKHOUSE_SU)
add_custom_target (clickhouse-su ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-su DEPENDS clickhouse)
install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-su" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
list(APPEND CLICKHOUSE_BUNDLE clickhouse-su)
endif ()
if (ENABLE_CLICKHOUSE_KEEPER)
if (NOT BUILD_STANDALONE_KEEPER AND CREATE_KEEPER_SYMLINK)

View File

@ -19,3 +19,4 @@
#cmakedefine01 ENABLE_CLICKHOUSE_KEEPER
#cmakedefine01 ENABLE_CLICKHOUSE_KEEPER_CONVERTER
#cmakedefine01 ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER
#cmakedefine01 ENABLE_CLICKHOUSE_SU

View File

@ -925,24 +925,7 @@ namespace
executable.string(), config.string(), pid_file.string());
if (!user.empty())
{
#if defined(OS_FREEBSD)
command = fmt::format("su -m '{}' -c '{}'", user, command);
#else
bool may_need_sudo = geteuid() != 0;
if (may_need_sudo)
{
struct passwd *p = getpwuid(geteuid());
// Only use sudo when we are not the given user
if (p == nullptr || std::string(p->pw_name) != user)
command = fmt::format("sudo -u '{}' {}", user, command);
}
else
{
command = fmt::format("su -s /bin/sh '{}' -c '{}'", user, command);
}
#endif
}
command = fmt::format("clickhouse su '{}' {}", user, command);
fmt::print("Will run {}\n", command);
executeScript(command, true);

View File

@ -65,6 +65,9 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv);
#if ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER
int mainEntryClickHouseStaticFilesDiskUploader(int argc, char ** argv);
#endif
#if ENABLE_CLICKHOUSE_SU
int mainEntryClickHouseSU(int argc, char ** argv);
#endif
#if ENABLE_CLICKHOUSE_INSTALL
int mainEntryClickHouseInstall(int argc, char ** argv);
int mainEntryClickHouseStart(int argc, char ** argv);
@ -81,8 +84,6 @@ int mainEntryClickHouseHashBinary(int, char **)
return 0;
}
#define ARRAY_SIZE(a) (sizeof(a)/sizeof((a)[0]))
namespace
{
@ -138,6 +139,9 @@ std::pair<const char *, MainFunc> clickhouse_applications[] =
#endif
#if ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER
{"static-files-disk-uploader", mainEntryClickHouseStaticFilesDiskUploader},
#endif
#if ENABLE_CLICKHOUSE_SU
{"su", mainEntryClickHouseSU},
#endif
{"hash-binary", mainEntryClickHouseHashBinary},
};
@ -189,7 +193,7 @@ auto instructionFailToString(InstructionFail fail)
{
switch (fail)
{
#define ret(x) return std::make_tuple(STDERR_FILENO, x, ARRAY_SIZE(x) - 1)
#define ret(x) return std::make_tuple(STDERR_FILENO, x, sizeof(x) - 1)
case InstructionFail::NONE:
ret("NONE");
case InstructionFail::SSE3:
@ -277,7 +281,7 @@ void checkRequiredInstructionsImpl(volatile InstructionFail & fail)
#define writeError(data) do \
{ \
static_assert(__builtin_constant_p(data)); \
if (!writeRetry(STDERR_FILENO, data, ARRAY_SIZE(data) - 1)) \
if (!writeRetry(STDERR_FILENO, data, sizeof(data) - 1)) \
_Exit(1); \
} while (false)
@ -334,6 +338,7 @@ struct Checker
#endif
;
/// NOTE: We will migrate to full static linking or our own dynamic loader to make this code obsolete.
void checkHarmfulEnvironmentVariables(char ** argv)
{
@ -406,17 +411,17 @@ int main(int argc_, char ** argv_)
inside_main = true;
SCOPE_EXIT({ inside_main = false; });
/// PHDR cache is required for query profiler to work reliably
/// It also speed up exception handling, but exceptions from dynamically loaded libraries (dlopen)
/// will work only after additional call of this function.
updatePHDRCache();
checkHarmfulEnvironmentVariables(argv_);
/// Reset new handler to default (that throws std::bad_alloc)
/// It is needed because LLVM library clobbers it.
std::set_new_handler(nullptr);
/// PHDR cache is required for query profiler to work reliably
/// It also speed up exception handling, but exceptions from dynamically loaded libraries (dlopen)
/// will work only after additional call of this function.
updatePHDRCache();
std::vector<char *> argv(argv_, argv_ + argc_);
/// Print a basic help if nothing was matched

View File

@ -200,6 +200,6 @@ try
}
catch (...)
{
std::cerr << DB::getCurrentExceptionMessage(false);
std::cerr << DB::getCurrentExceptionMessage(false) << '\n';
return 1;
}

View File

@ -0,0 +1,3 @@
set (CLICKHOUSE_SU_SOURCES clickhouse-su.cpp)
set (CLICKHOUSE_SU_LINK PRIVATE dbms)
clickhouse_program_add(su)

View File

@ -0,0 +1,145 @@
#include <Common/Exception.h>
#include <IO/ReadHelpers.h>
#include <fmt/format.h>
#include <vector>
#include <sys/types.h>
#include <unistd.h>
#include <pwd.h>
#include <grp.h>
/// "su" means "set user"
/// In fact, this program can set Unix user and group.
///
/// Usage:
/// clickhouse su user[:group] args...
///
/// - will set user and, optionally, group and exec the remaining args.
/// user and group can be numeric identifiers or strings.
///
/// The motivation for this tool is very obscure and idiosyncratic. It is needed for Docker.
/// People want to run programs inside Docker with dropped privileges (less than root).
/// But the standard Linux "su" program is not suitable for usage inside Docker,
/// because it is creating pseudoterminals to avoid hijacking input from the terminal, for security,
/// but Docker is also doing something with the terminal and it is incompatible.
/// For this reason, people use alternative and less "secure" versions of "su" tools like "gosu" or "su-exec".
/// But it would be very strange to use 3rd-party software only to do two-three syscalls.
/// That's why we provide this tool.
///
/// Note: ClickHouse does not need Docker at all and works better without Docker.
/// ClickHouse has no dependencies, it is packaged and distributed in single binary.
/// There is no reason to use Docker unless you are already running all your software in Docker.
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int SYSTEM_ERROR;
}
void setUserAndGroup(std::string arg_uid, std::string arg_gid)
{
static constexpr size_t buf_size = 16384; /// Linux man page says it is enough. Nevertheless, we will check if it's not enough and throw.
std::unique_ptr<char[]> buf(new char[buf_size]);
/// Set the group first, because if we set user, the privileges will be already dropped and we will not be able to set the group later.
if (!arg_gid.empty())
{
gid_t gid = 0;
if (!tryParse(gid, arg_gid) || gid == 0)
{
group entry{};
group * result{};
if (0 != getgrnam_r(arg_gid.data(), &entry, buf.get(), buf_size, &result))
throwFromErrno(fmt::format("Cannot do 'getgrnam_r' to obtain gid from group name ({})", arg_gid), ErrorCodes::SYSTEM_ERROR);
if (!result)
throw Exception("Group {} is not found in the system", ErrorCodes::BAD_ARGUMENTS);
gid = entry.gr_gid;
}
if (gid == 0 && getgid() != 0)
throw Exception("Group has id 0, but dropping privileges to gid 0 does not make sense", ErrorCodes::BAD_ARGUMENTS);
if (0 != setgid(gid))
throwFromErrno(fmt::format("Cannot do 'setgid' to user ({})", arg_gid), ErrorCodes::SYSTEM_ERROR);
}
if (!arg_uid.empty())
{
/// Is it numeric id or name?
uid_t uid = 0;
if (!tryParse(uid, arg_uid) || uid == 0)
{
passwd entry{};
passwd * result{};
if (0 != getpwnam_r(arg_uid.data(), &entry, buf.get(), buf_size, &result))
throwFromErrno(fmt::format("Cannot do 'getpwnam_r' to obtain uid from user name ({})", arg_uid), ErrorCodes::SYSTEM_ERROR);
if (!result)
throw Exception("User {} is not found in the system", ErrorCodes::BAD_ARGUMENTS);
uid = entry.pw_uid;
}
if (uid == 0 && getuid() != 0)
throw Exception("User has id 0, but dropping privileges to uid 0 does not make sense", ErrorCodes::BAD_ARGUMENTS);
if (0 != setuid(uid))
throwFromErrno(fmt::format("Cannot do 'setuid' to user ({})", arg_uid), ErrorCodes::SYSTEM_ERROR);
}
}
}
int mainEntryClickHouseSU(int argc, char ** argv)
try
{
using namespace DB;
if (argc < 3)
{
std::cout << "Usage: ./clickhouse su user:group ..." << std::endl;
exit(0);
}
std::string_view user_and_group = argv[1];
std::string user;
std::string group;
auto pos = user_and_group.find(':');
if (pos == std::string_view::npos)
{
user = user_and_group;
}
else
{
user = user_and_group.substr(0, pos);
group = user_and_group.substr(pos + 1);
}
setUserAndGroup(std::move(user), std::move(group));
std::vector<char *> new_argv;
new_argv.reserve(argc - 1);
new_argv.insert(new_argv.begin(), argv + 2, argv + argc);
new_argv.push_back(nullptr);
execvp(new_argv.front(), new_argv.data());
throwFromErrno("Cannot execvp", ErrorCodes::SYSTEM_ERROR);
}
catch (...)
{
std::cerr << DB::getCurrentExceptionMessage(false) << '\n';
return 1;
}

View File

@ -17,7 +17,7 @@ namespace DB
template <bool thread_safe>
void OptimizedRegularExpressionImpl<thread_safe>::analyze(
const std::string & regexp,
std::string_view regexp,
std::string & required_substring,
bool & is_trivial,
bool & required_substring_is_prefix)

View File

@ -86,8 +86,6 @@ public:
/// Get the regexp re2 or nullptr if the pattern is trivial (for output to the log).
const std::unique_ptr<RegexType> & getRE2() const { return re2; }
static void analyze(const std::string & regexp_, std::string & required_substring, bool & is_trivial, bool & required_substring_is_prefix);
void getAnalyzeResult(std::string & out_required_substring, bool & out_is_trivial, bool & out_required_substring_is_prefix) const
{
out_required_substring = required_substring;
@ -104,6 +102,8 @@ private:
std::optional<DB::StringSearcher<false, true>> case_insensitive_substring_searcher;
std::unique_ptr<RegexType> re2;
unsigned number_of_subpatterns;
static void analyze(std::string_view regexp_, std::string & required_substring, bool & is_trivial, bool & required_substring_is_prefix);
};
using OptimizedRegularExpression = OptimizedRegularExpressionImpl<true>;

View File

@ -15,7 +15,6 @@
#include <IO/WriteHelpers.h>
#include <boost/algorithm/string.hpp>
#include <libnuraft/cluster_config.hxx>
#include <libnuraft/log_val_type.hxx>
#include <libnuraft/raft_server.hxx>
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/Util/Application.h>
@ -316,22 +315,6 @@ void KeeperServer::startup(const Poco::Util::AbstractConfiguration & config, boo
state_manager->loadLogStore(state_machine->last_commit_index() + 1, coordination_settings->reserved_log_items);
auto log_store = state_manager->load_log_store();
auto next_log_idx = log_store->next_slot();
if (next_log_idx > 0 && next_log_idx > state_machine->last_commit_index())
{
auto log_entries = log_store->log_entries(state_machine->last_commit_index() + 1, next_log_idx);
auto idx = state_machine->last_commit_index() + 1;
for (const auto & entry : *log_entries)
{
if (entry && entry->get_val_type() == nuraft::log_val_type::app_log)
state_machine->preprocess(idx, entry->get_buf());
++idx;
}
}
loadLatestConfig();
last_local_config = state_manager->parseServersConfiguration(config, true).cluster_config;

View File

@ -44,6 +44,7 @@ namespace
else /// backward compatibility
request_for_session.time = std::chrono::duration_cast<std::chrono::milliseconds>(std::chrono::system_clock::now().time_since_epoch()).count();
return request_for_session;
}
}
@ -113,21 +114,6 @@ void KeeperStateMachine::init()
storage = std::make_unique<KeeperStorage>(coordination_settings->dead_session_check_period_ms.totalMilliseconds(), superdigest);
}
nuraft::ptr<nuraft::buffer> KeeperStateMachine::pre_commit(uint64_t log_idx, nuraft::buffer & data)
{
preprocess(log_idx, data);
return nullptr;
}
void KeeperStateMachine::preprocess(const uint64_t log_idx, nuraft::buffer & data)
{
auto request_for_session = parseRequest(data);
if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID)
return;
std::lock_guard lock(storage_and_responses_lock);
storage->preprocessRequest(request_for_session.request, request_for_session.session_id, request_for_session.time, log_idx);
}
nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data)
{
auto request_for_session = parseRequest(data);
@ -196,12 +182,6 @@ void KeeperStateMachine::commit_config(const uint64_t /* log_idx */, nuraft::ptr
cluster_config = ClusterConfig::deserialize(*tmp);
}
void KeeperStateMachine::rollback(uint64_t log_idx, nuraft::buffer & /*data*/)
{
std::lock_guard lock(storage_and_responses_lock);
storage->rollbackRequest(log_idx);
}
nuraft::ptr<nuraft::snapshot> KeeperStateMachine::last_snapshot()
{
/// Just return the latest snapshot.
@ -363,7 +343,7 @@ void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSessi
{
/// Pure local request, just process it with storage
std::lock_guard lock(storage_and_responses_lock);
auto responses = storage->processRequest(request_for_session.request, request_for_session.session_id, request_for_session.time, std::nullopt, true /*check_acl*/, true /*is_local*/);
auto responses = storage->processRequest(request_for_session.request, request_for_session.session_id, request_for_session.time, std::nullopt);
for (const auto & response : responses)
if (!responses_queue.push(response))
throw Exception(ErrorCodes::SYSTEM_ERROR, "Could not push response with session id {} into responses queue", response.session_id);

View File

@ -27,16 +27,16 @@ public:
/// Read state from the latest snapshot
void init();
void preprocess(uint64_t log_idx, nuraft::buffer & data);
nuraft::ptr<nuraft::buffer> pre_commit(uint64_t log_idx, nuraft::buffer & data) override;
/// Currently not supported
nuraft::ptr<nuraft::buffer> pre_commit(const uint64_t /*log_idx*/, nuraft::buffer & /*data*/) override { return nullptr; }
nuraft::ptr<nuraft::buffer> commit(const uint64_t log_idx, nuraft::buffer & data) override; /// NOLINT
/// Save new cluster config to our snapshot (copy of the config stored in StateManager)
void commit_config(const uint64_t log_idx, nuraft::ptr<nuraft::cluster_config> & new_conf) override; /// NOLINT
void rollback(uint64_t log_idx, nuraft::buffer & data) override;
/// Currently not supported
void rollback(const uint64_t /*log_idx*/, nuraft::buffer & /*data*/) override {}
uint64_t last_commit_index() override { return last_committed_idx; }

File diff suppressed because it is too large Load Diff

View File

@ -1,14 +1,14 @@
#pragma once
#include <unordered_map>
#include <vector>
#include <Coordination/ACLMap.h>
#include <Common/ZooKeeper/IKeeper.h>
#include <Common/ConcurrentBoundedQueue.h>
#include <Common/ZooKeeper/ZooKeeperCommon.h>
#include <Coordination/SessionExpiryQueue.h>
#include <Coordination/ACLMap.h>
#include <Coordination/SnapshotableHashTable.h>
#include <IO/WriteBufferFromString.h>
#include <Common/ConcurrentBoundedQueue.h>
#include <Common/ZooKeeper/IKeeper.h>
#include <Common/ZooKeeper/ZooKeeperCommon.h>
#include <unordered_map>
#include <vector>
#include <absl/container/flat_hash_set.h>
@ -29,6 +29,7 @@ struct KeeperStorageSnapshot;
class KeeperStorage
{
public:
struct Node
{
uint64_t acl_id = 0; /// 0 -- no ACL by default
@ -40,18 +41,26 @@ public:
Node() : size_bytes(sizeof(Node)) { }
/// Object memory size
uint64_t sizeInBytes() const { return size_bytes; }
uint64_t sizeInBytes() const
{
return size_bytes;
}
void setData(String new_data);
const auto & getData() const noexcept { return data; }
const auto & getData() const noexcept
{
return data;
}
void addChild(StringRef child_path);
void removeChild(StringRef child_path);
const auto & getChildren() const noexcept { return children; }
const auto & getChildren() const noexcept
{
return children;
}
private:
String data;
ChildrenSet children{};
@ -76,7 +85,10 @@ public:
std::string scheme;
std::string id;
bool operator==(const AuthID & other) const { return scheme == other.scheme && id == other.id; }
bool operator==(const AuthID & other) const
{
return scheme == other.scheme && id == other.id;
}
};
using RequestsForSessions = std::vector<RequestForSession>;
@ -100,146 +112,6 @@ public:
/// container.
Container container;
// Applying ZooKeeper request to storage consists of two steps:
// - preprocessing which, instead of applying the changes directly to storage,
// generates deltas with those changes, denoted with the request ZXID
// - processing which applies deltas with the correct ZXID to the storage
//
// Delta objects allow us two things:
// - fetch the latest, uncommitted state of an object by getting the committed
// state of that same object from the storage and applying the deltas
// in the same order as they are defined
// - quickly commit the changes to the storage
struct CreateNodeDelta
{
Coordination::Stat stat;
bool is_ephemeral;
bool is_sequental;
Coordination::ACLs acls;
String data;
};
struct RemoveNodeDelta
{
int32_t version{-1};
};
struct UpdateNodeDelta
{
std::function<void(Node &)> update_fn;
int32_t version{-1};
};
struct SetACLDelta
{
Coordination::ACLs acls;
int32_t version{-1};
};
struct ErrorDelta
{
Coordination::Error error;
};
struct FailedMultiDelta
{
std::vector<Coordination::Error> error_codes;
};
// Denotes end of a subrequest in multi request
struct SubDeltaEnd
{
};
struct AddAuthDelta
{
int64_t session_id;
AuthID auth_id;
};
using Operation
= std::variant<CreateNodeDelta, RemoveNodeDelta, UpdateNodeDelta, SetACLDelta, AddAuthDelta, ErrorDelta, SubDeltaEnd, FailedMultiDelta>;
struct Delta
{
Delta(String path_, int64_t zxid_, Operation operation_) : path(std::move(path_)), zxid(zxid_), operation(std::move(operation_)) { }
Delta(int64_t zxid_, Coordination::Error error) : Delta("", zxid_, ErrorDelta{error}) { }
Delta(int64_t zxid_, Operation subdelta) : Delta("", zxid_, subdelta) { }
String path;
int64_t zxid;
Operation operation;
};
struct UncommittedState
{
explicit UncommittedState(KeeperStorage & storage_) : storage(storage_) { }
template <typename Visitor>
void applyDeltas(StringRef path, const Visitor & visitor) const
{
for (const auto & delta : deltas)
{
if (path.empty() || delta.path == path)
std::visit(visitor, delta.operation);
}
}
bool hasACL(int64_t session_id, bool is_local, std::function<bool(const AuthID &)> predicate)
{
for (const auto & session_auth : storage.session_and_auth[session_id])
{
if (predicate(session_auth))
return true;
}
if (is_local)
return false;
for (const auto & delta : deltas)
{
if (const auto * auth_delta = std::get_if<KeeperStorage::AddAuthDelta>(&delta.operation);
auth_delta && auth_delta->session_id == session_id && predicate(auth_delta->auth_id))
return true;
}
return false;
}
std::shared_ptr<Node> getNode(StringRef path);
bool hasNode(StringRef path) const;
Coordination::ACLs getACLs(StringRef path) const;
std::deque<Delta> deltas;
KeeperStorage & storage;
};
UncommittedState uncommitted_state{*this};
Coordination::Error commit(int64_t zxid, int64_t session_id);
// Create node in the storage
// Returns false if it failed to create the node, true otherwise
// We don't care about the exact failure because we should've caught it during preprocessing
bool createNode(
const std::string & path,
String data,
const Coordination::Stat & stat,
bool is_sequental,
bool is_ephemeral,
Coordination::ACLs node_acls,
int64_t session_id);
// Remove node in the storage
// Returns false if it failed to remove the node, true otherwise
// We don't care about the exact failure because we should've caught it during preprocessing
bool removeNode(const std::string & path, int32_t version);
bool checkACL(StringRef path, int32_t permissions, int64_t session_id, bool is_local);
/// Mapping session_id -> set of ephemeral nodes paths
Ephemerals ephemerals;
/// Mapping session_id -> set of watched nodes paths
@ -258,12 +130,15 @@ public:
/// Currently active watches (node_path -> subscribed sessions)
Watches watches;
Watches list_watches; /// Watches for 'list' request (watches on children).
Watches list_watches; /// Watches for 'list' request (watches on children).
void clearDeadWatches(int64_t session_id);
/// Get current zxid
int64_t getZXID() const { return zxid; }
int64_t getZXID() const
{
return zxid;
}
const String superdigest;
@ -287,53 +162,78 @@ public:
/// Process user request and return response.
/// check_acl = false only when converting data from ZooKeeper.
ResponsesForSessions processRequest(
const Coordination::ZooKeeperRequestPtr & request,
int64_t session_id,
int64_t time,
std::optional<int64_t> new_last_zxid,
bool check_acl = true,
bool is_local = false);
void preprocessRequest(
const Coordination::ZooKeeperRequestPtr & request, int64_t session_id, int64_t time, int64_t new_last_zxid, bool check_acl = true);
void rollbackRequest(int64_t rollback_zxid);
ResponsesForSessions processRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id, int64_t time, std::optional<int64_t> new_last_zxid, bool check_acl = true);
void finalize();
/// Set of methods for creating snapshots
/// Turn on snapshot mode, so data inside Container is not deleted, but replaced with new version.
void enableSnapshotMode(size_t up_to_version) { container.enableSnapshotMode(up_to_version); }
void enableSnapshotMode(size_t up_to_version)
{
container.enableSnapshotMode(up_to_version);
}
/// Turn off snapshot mode.
void disableSnapshotMode() { container.disableSnapshotMode(); }
void disableSnapshotMode()
{
container.disableSnapshotMode();
}
Container::const_iterator getSnapshotIteratorBegin() const { return container.begin(); }
Container::const_iterator getSnapshotIteratorBegin() const
{
return container.begin();
}
/// Clear outdated data from internal container.
void clearGarbageAfterSnapshot() { container.clearOutdatedNodes(); }
void clearGarbageAfterSnapshot()
{
container.clearOutdatedNodes();
}
/// Get all active sessions
const SessionAndTimeout & getActiveSessions() const { return session_and_timeout; }
const SessionAndTimeout & getActiveSessions() const
{
return session_and_timeout;
}
/// Get all dead sessions
std::vector<int64_t> getDeadSessions() const { return session_expiry_queue.getExpiredSessions(); }
std::vector<int64_t> getDeadSessions() const
{
return session_expiry_queue.getExpiredSessions();
}
/// Introspection functions mostly used in 4-letter commands
uint64_t getNodesCount() const { return container.size(); }
uint64_t getNodesCount() const
{
return container.size();
}
uint64_t getApproximateDataSize() const { return container.getApproximateDataSize(); }
uint64_t getApproximateDataSize() const
{
return container.getApproximateDataSize();
}
uint64_t getArenaDataSize() const { return container.keyArenaSize(); }
uint64_t getArenaDataSize() const
{
return container.keyArenaSize();
}
uint64_t getTotalWatchesCount() const;
uint64_t getWatchedPathsCount() const { return watches.size() + list_watches.size(); }
uint64_t getWatchedPathsCount() const
{
return watches.size() + list_watches.size();
}
uint64_t getSessionsWithWatchesCount() const;
uint64_t getSessionWithEphemeralNodesCount() const { return ephemerals.size(); }
uint64_t getSessionWithEphemeralNodesCount() const
{
return ephemerals.size();
}
uint64_t getTotalEphemeralNodesCount() const;
void dumpWatches(WriteBufferFromOwnString & buf) const;

View File

@ -12,6 +12,7 @@ public:
WriteBufferFromNuraftBuffer();
nuraft::ptr<nuraft::buffer> getBuffer();
bool isFinished() const { return finalized; }
~WriteBufferFromNuraftBuffer() override;

View File

@ -520,7 +520,6 @@ bool deserializeTxn(KeeperStorage & storage, ReadBuffer & in, Poco::Logger * /*l
if (request->getOpNum() == Coordination::OpNum::Multi && hasErrorsInMultiRequest(request))
return true;
storage.preprocessRequest(request, session_id, time, zxid, /* check_acl = */ false);
storage.processRequest(request, session_id, time, zxid, /* check_acl = */ false);
}
}

View File

@ -1,8 +1,6 @@
#include <chrono>
#include <gtest/gtest.h>
#include "Common/ZooKeeper/IKeeper.h"
#include "Coordination/KeeperStorage.h"
#include "config_core.h"
#if USE_NURAFT
@ -1263,7 +1261,6 @@ void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, uint
changelog.append(entry);
changelog.end_of_append_batch(0, 0);
state_machine->pre_commit(i, changelog.entry_at(i)->get_buf());
state_machine->commit(i, changelog.entry_at(i)->get_buf());
bool snapshot_created = false;
if (i % settings->snapshot_distance == 0)
@ -1308,7 +1305,6 @@ void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, uint
for (size_t i = restore_machine->last_commit_index() + 1; i < restore_changelog.next_slot(); ++i)
{
restore_machine->pre_commit(i, changelog.entry_at(i)->get_buf());
restore_machine->commit(i, changelog.entry_at(i)->get_buf());
}
@ -1411,7 +1407,6 @@ TEST_P(CoordinationTest, TestEphemeralNodeRemove)
request_c->path = "/hello";
request_c->is_ephemeral = true;
auto entry_c = getLogEntryFromZKRequest(0, 1, request_c);
state_machine->pre_commit(1, entry_c->get_buf());
state_machine->commit(1, entry_c->get_buf());
const auto & storage = state_machine->getStorage();
@ -1420,7 +1415,6 @@ TEST_P(CoordinationTest, TestEphemeralNodeRemove)
request_d->path = "/hello";
/// Delete from other session
auto entry_d = getLogEntryFromZKRequest(0, 2, request_d);
state_machine->pre_commit(2, entry_d->get_buf());
state_machine->commit(2, entry_d->get_buf());
EXPECT_EQ(storage.ephemerals.size(), 0);
@ -1783,130 +1777,6 @@ TEST_P(CoordinationTest, TestLogGap)
EXPECT_EQ(changelog1.next_slot(), 61);
}
template <typename ResponseType>
ResponseType getSingleResponse(const auto & responses)
{
EXPECT_FALSE(responses.empty());
return dynamic_cast<ResponseType &>(*responses[0].response);
}
TEST_P(CoordinationTest, TestUncommittedStateBasicCrud)
{
using namespace DB;
using namespace Coordination;
DB::KeeperStorage storage{500, ""};
constexpr std::string_view path = "/test";
const auto get_committed_data = [&]() -> std::optional<String>
{
auto request = std::make_shared<ZooKeeperGetRequest>();
request->path = path;
auto responses = storage.processRequest(request, 0, 0, std::nullopt, true, true);
const auto & get_response = getSingleResponse<ZooKeeperGetResponse>(responses);
if (get_response.error != Error::ZOK)
return std::nullopt;
return get_response.data;
};
const auto preprocess_get = [&](int64_t zxid)
{
auto get_request = std::make_shared<ZooKeeperGetRequest>();
get_request->path = path;
storage.preprocessRequest(get_request, 0, 0, zxid);
return get_request;
};
const auto create_request = std::make_shared<ZooKeeperCreateRequest>();
create_request->path = path;
create_request->data = "initial_data";
storage.preprocessRequest(create_request, 0, 0, 1);
storage.preprocessRequest(create_request, 0, 0, 2);
ASSERT_FALSE(get_committed_data());
const auto after_create_get = preprocess_get(3);
ASSERT_FALSE(get_committed_data());
const auto set_request = std::make_shared<ZooKeeperSetRequest>();
set_request->path = path;
set_request->data = "new_data";
storage.preprocessRequest(set_request, 0, 0, 4);
const auto after_set_get = preprocess_get(5);
ASSERT_FALSE(get_committed_data());
const auto remove_request = std::make_shared<ZooKeeperRemoveRequest>();
remove_request->path = path;
storage.preprocessRequest(remove_request, 0, 0, 6);
storage.preprocessRequest(remove_request, 0, 0, 7);
const auto after_remove_get = preprocess_get(8);
ASSERT_FALSE(get_committed_data());
{
const auto responses = storage.processRequest(create_request, 0, 0, 1);
const auto & create_response = getSingleResponse<ZooKeeperCreateResponse>(responses);
ASSERT_EQ(create_response.error, Error::ZOK);
}
{
const auto responses = storage.processRequest(create_request, 0, 0, 2);
const auto & create_response = getSingleResponse<ZooKeeperCreateResponse>(responses);
ASSERT_EQ(create_response.error, Error::ZNODEEXISTS);
}
{
const auto responses = storage.processRequest(after_create_get, 0, 0, 3);
const auto & get_response = getSingleResponse<ZooKeeperGetResponse>(responses);
ASSERT_EQ(get_response.error, Error::ZOK);
ASSERT_EQ(get_response.data, "initial_data");
}
ASSERT_EQ(get_committed_data(), "initial_data");
{
const auto responses = storage.processRequest(set_request, 0, 0, 4);
const auto & create_response = getSingleResponse<ZooKeeperSetResponse>(responses);
ASSERT_EQ(create_response.error, Error::ZOK);
}
{
const auto responses = storage.processRequest(after_set_get, 0, 0, 5);
const auto & get_response = getSingleResponse<ZooKeeperGetResponse>(responses);
ASSERT_EQ(get_response.error, Error::ZOK);
ASSERT_EQ(get_response.data, "new_data");
}
ASSERT_EQ(get_committed_data(), "new_data");
{
const auto responses = storage.processRequest(remove_request, 0, 0, 6);
const auto & create_response = getSingleResponse<ZooKeeperRemoveResponse>(responses);
ASSERT_EQ(create_response.error, Error::ZOK);
}
{
const auto responses = storage.processRequest(remove_request, 0, 0, 7);
const auto & create_response = getSingleResponse<ZooKeeperRemoveResponse>(responses);
ASSERT_EQ(create_response.error, Error::ZNONODE);
}
{
const auto responses = storage.processRequest(after_remove_get, 0, 0, 8);
const auto & get_response = getSingleResponse<ZooKeeperGetResponse>(responses);
ASSERT_EQ(get_response.error, Error::ZNONODE);
}
ASSERT_FALSE(get_committed_data());
}
INSTANTIATE_TEST_SUITE_P(CoordinationTestSuite,
CoordinationTest,

View File

@ -16,7 +16,7 @@ class SerializationNothing : public SimpleTextSerialization
private:
[[noreturn]] static void throwNoSerialization()
{
throw Exception("Serialization is not implemented", ErrorCodes::NOT_IMPLEMENTED);
throw Exception("Serialization is not implemented for type Nothing", ErrorCodes::NOT_IMPLEMENTED);
}
public:
void serializeBinary(const Field &, WriteBuffer &) const override { throwNoSerialization(); }

View File

@ -252,7 +252,7 @@ Strings DictionaryStructure::getKeysNames() const
static void checkAttributeKeys(const Poco::Util::AbstractConfiguration::Keys & keys)
{
static const std::unordered_set<std::string_view> valid_keys
= {"name", "type", "expression", "null_value", "hierarchical", "injective", "is_object_id"};
= {"name", "type", "expression", "null_value", "hierarchical", "bidirectional", "injective", "is_object_id"};
for (const auto & key : keys)
{
@ -350,6 +350,7 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
}
const auto hierarchical = config.getBool(prefix + "hierarchical", false);
const auto bidirectional = config.getBool(prefix + "bidirectional", false);
const auto injective = config.getBool(prefix + "injective", false);
const auto is_object_id = config.getBool(prefix + "is_object_id", false);
@ -362,6 +363,9 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
if (has_hierarchy && hierarchical)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Only one hierarchical attribute supported");
if (bidirectional && !hierarchical)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bidirectional can only be applied to hierarchical attributes");
has_hierarchy = has_hierarchy || hierarchical;
res_attributes.emplace_back(DictionaryAttribute{
@ -372,6 +376,7 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
expression,
null_value,
hierarchical,
bidirectional,
injective,
is_object_id,
is_nullable});

View File

@ -67,6 +67,7 @@ struct DictionaryAttribute final
const std::string expression;
const Field null_value;
const bool hierarchical;
const bool bidirectional;
const bool injective;
const bool is_object_id;
const bool is_nullable;

View File

@ -43,6 +43,7 @@ FlatDictionary::FlatDictionary(
{
createAttributes();
loadData();
buildHierarchyParentToChildIndexIfNeeded();
calculateBytesAllocated();
}
@ -244,30 +245,43 @@ ColumnUInt8::Ptr FlatDictionary::isInHierarchy(
return result;
}
ColumnPtr FlatDictionary::getDescendants(
ColumnPtr key_column,
const DataTypePtr &,
size_t level) const
DictionaryHierarchyParentToChildIndexPtr FlatDictionary::getHierarchicalIndex() const
{
PaddedPODArray<UInt64> keys_backup;
const auto & keys = getColumnVectorData(this, key_column, keys_backup);
if (hierarhical_index)
return hierarhical_index;
size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index;
const auto & hierarchical_attribute = attributes[hierarchical_attribute_index];
const ContainerType<UInt64> & parent_keys = std::get<ContainerType<UInt64>>(hierarchical_attribute.container);
HashMap<UInt64, PaddedPODArray<UInt64>> parent_to_child;
parent_to_child.reserve(element_count);
for (size_t i = 0; i < parent_keys.size(); ++i)
UInt64 child_keys_size = static_cast<UInt64>(parent_keys.size());
for (UInt64 child_key = 0; child_key < child_keys_size; ++child_key)
{
auto parent_key = parent_keys[i];
if (!loaded_keys[child_key])
continue;
if (loaded_keys[i])
parent_to_child[parent_key].emplace_back(static_cast<UInt64>(i));
auto parent_key = parent_keys[child_key];
parent_to_child[parent_key].emplace_back(child_key);
}
return std::make_shared<DictionaryHierarchicalParentToChildIndex>(parent_to_child);
}
ColumnPtr FlatDictionary::getDescendants(
ColumnPtr key_column,
const DataTypePtr &,
size_t level,
DictionaryHierarchicalParentToChildIndexPtr parent_to_child_index) const
{
PaddedPODArray<UInt64> keys_backup;
const auto & keys = getColumnVectorData(this, key_column, keys_backup);
size_t keys_found;
auto result = getKeysDescendantsArray(keys, parent_to_child, level, keys_found);
auto result = getKeysDescendantsArray(keys, *parent_to_child_index, level, keys_found);
query_count.fetch_add(keys.size(), std::memory_order_relaxed);
found_count.fetch_add(keys_found, std::memory_order_relaxed);
@ -400,6 +414,15 @@ void FlatDictionary::loadData()
throw Exception(ErrorCodes::DICTIONARY_IS_EMPTY, "{}: dictionary source is empty and 'require_nonempty' property is set.", getFullName());
}
void FlatDictionary::buildHierarchyParentToChildIndexIfNeeded()
{
if (!dict_struct.hierarchical_attribute_index)
return;
if (dict_struct.attributes[*dict_struct.hierarchical_attribute_index].bidirectional)
hierarhical_index = getHierarchicalIndex();
}
void FlatDictionary::calculateBytesAllocated()
{
bytes_allocated += attributes.size() * sizeof(attributes.front());
@ -439,6 +462,12 @@ void FlatDictionary::calculateBytesAllocated()
if (update_field_loaded_block)
bytes_allocated += update_field_loaded_block->allocatedBytes();
if (hierarhical_index)
{
hierarchical_index_bytes_allocated = hierarhical_index->getSizeInBytes();
bytes_allocated += hierarchical_index_bytes_allocated;
}
bytes_allocated += string_arena.size();
}

View File

@ -92,10 +92,15 @@ public:
ColumnPtr in_key_column,
const DataTypePtr & key_type) const override;
DictionaryHierarchicalParentToChildIndexPtr getHierarchicalIndex() const override;
size_t getHierarchicalIndexBytesAllocated() const override { return hierarchical_index_bytes_allocated; }
ColumnPtr getDescendants(
ColumnPtr key_column,
const DataTypePtr & key_type,
size_t level) const override;
size_t level,
DictionaryHierarchicalParentToChildIndexPtr parent_to_child_index) const override;
Pipe read(const Names & column_names, size_t max_block_size, size_t num_streams) const override;
@ -137,10 +142,15 @@ private:
};
void createAttributes();
void blockToAttributes(const Block & block);
void updateData();
void loadData();
void buildHierarchyParentToChildIndexIfNeeded();
void calculateBytesAllocated();
Attribute createAttribute(const DictionaryAttribute & attribute);
@ -165,6 +175,7 @@ private:
std::vector<bool> loaded_keys;
size_t bytes_allocated = 0;
size_t hierarchical_index_bytes_allocated = 0;
size_t element_count = 0;
size_t bucket_count = 0;
mutable std::atomic<size_t> query_count{0};
@ -172,6 +183,7 @@ private:
BlockPtr update_field_loaded_block;
Arena string_arena;
DictionaryHierarchicalParentToChildIndexPtr hierarhical_index;
};
}

View File

@ -37,6 +37,7 @@ HashedArrayDictionary<dictionary_key_type>::HashedArrayDictionary(
{
createAttributes();
loadData();
buildHierarchyParentToChildIndexIfNeeded();
calculateBytesAllocated();
}
@ -282,18 +283,14 @@ ColumnUInt8::Ptr HashedArrayDictionary<dictionary_key_type>::isInHierarchy(
}
template <DictionaryKeyType dictionary_key_type>
ColumnPtr HashedArrayDictionary<dictionary_key_type>::getDescendants(
ColumnPtr key_column [[maybe_unused]],
const DataTypePtr &,
size_t level [[maybe_unused]]) const
DictionaryHierarchicalParentToChildIndexPtr HashedArrayDictionary<dictionary_key_type>::getHierarchicalIndex() const
{
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{
PaddedPODArray<UInt64> keys_backup;
const auto & keys = getColumnVectorData(this, key_column, keys_backup);
if (hierarchical_index)
return hierarchical_index;
size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index;
const auto & hierarchical_attribute = attributes[hierarchical_attribute_index];
const AttributeContainerType<UInt64> & parent_keys_container = std::get<AttributeContainerType<UInt64>>(hierarchical_attribute.container);
@ -306,6 +303,7 @@ ColumnPtr HashedArrayDictionary<dictionary_key_type>::getDescendants(
index_to_key[value] = key;
HashMap<UInt64, PaddedPODArray<UInt64>> parent_to_child;
parent_to_child.reserve(index_to_key.size());
for (size_t i = 0; i < parent_keys_container.size(); ++i)
{
@ -313,13 +311,33 @@ ColumnPtr HashedArrayDictionary<dictionary_key_type>::getDescendants(
if (it == index_to_key.end())
continue;
auto parent_key = it->getMapped();
auto child_key = parent_keys_container[i];
auto child_key = it->getMapped();
auto parent_key = parent_keys_container[i];
parent_to_child[parent_key].emplace_back(child_key);
}
return std::make_shared<DictionaryHierarchicalParentToChildIndex>(parent_to_child);
}
else
{
return nullptr;
}
}
template <DictionaryKeyType dictionary_key_type>
ColumnPtr HashedArrayDictionary<dictionary_key_type>::getDescendants(
ColumnPtr key_column [[maybe_unused]],
const DataTypePtr &,
size_t level [[maybe_unused]],
DictionaryHierarchicalParentToChildIndexPtr parent_to_child_index [[maybe_unused]]) const
{
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{
PaddedPODArray<UInt64> keys_backup;
const auto & keys = getColumnVectorData(this, key_column, keys_backup);
size_t keys_found = 0;
auto result = getKeysDescendantsArray(keys, parent_to_child, level, keys_found);
auto result = getKeysDescendantsArray(keys, *parent_to_child_index, level, keys_found);
query_count.fetch_add(keys.size(), std::memory_order_relaxed);
found_count.fetch_add(keys_found, std::memory_order_relaxed);
@ -693,6 +711,16 @@ void HashedArrayDictionary<dictionary_key_type>::loadData()
getFullName());
}
template <DictionaryKeyType dictionary_key_type>
void HashedArrayDictionary<dictionary_key_type>::buildHierarchyParentToChildIndexIfNeeded()
{
if (!dict_struct.hierarchical_attribute_index)
return;
if (dict_struct.attributes[*dict_struct.hierarchical_attribute_index].bidirectional)
hierarchical_index = getHierarchicalIndex();
}
template <DictionaryKeyType dictionary_key_type>
void HashedArrayDictionary<dictionary_key_type>::calculateBytesAllocated()
{
@ -730,10 +758,16 @@ void HashedArrayDictionary<dictionary_key_type>::calculateBytesAllocated()
bytes_allocated += (*attribute.is_index_null).size();
}
bytes_allocated += string_arena.size();
if (update_field_loaded_block)
bytes_allocated += update_field_loaded_block->allocatedBytes();
if (hierarchical_index)
{
hierarchical_index_bytes_allocated = hierarchical_index->getSizeInBytes();
bytes_allocated += hierarchical_index_bytes_allocated;
}
bytes_allocated += string_arena.size();
}
template <DictionaryKeyType dictionary_key_type>

View File

@ -109,10 +109,15 @@ public:
ColumnPtr in_key_column,
const DataTypePtr & key_type) const override;
DictionaryHierarchicalParentToChildIndexPtr getHierarchicalIndex() const override;
size_t getHierarchicalIndexBytesAllocated() const override { return hierarchical_index_bytes_allocated; }
ColumnPtr getDescendants(
ColumnPtr key_column,
const DataTypePtr & key_type,
size_t level) const override;
size_t level,
DictionaryHierarchicalParentToChildIndexPtr parent_to_child_index) const override;
Pipe read(const Names & column_names, size_t max_block_size, size_t num_streams) const override;
@ -173,6 +178,8 @@ private:
void loadData();
void buildHierarchyParentToChildIndexIfNeeded();
void calculateBytesAllocated();
template <typename KeysProvider>
@ -214,6 +221,7 @@ private:
KeyAttribute key_attribute;
size_t bytes_allocated = 0;
size_t hierarchical_index_bytes_allocated = 0;
size_t element_count = 0;
size_t bucket_count = 0;
mutable std::atomic<size_t> query_count{0};
@ -221,6 +229,7 @@ private:
BlockPtr update_field_loaded_block;
Arena string_arena;
DictionaryHierarchicalParentToChildIndexPtr hierarchical_index;
};
extern template class HashedArrayDictionary<DictionaryKeyType::Simple>;

View File

@ -54,6 +54,7 @@ HashedDictionary<dictionary_key_type, sparse>::HashedDictionary(
{
createAttributes();
loadData();
buildHierarchyParentToChildIndexIfNeeded();
calculateBytesAllocated();
}
@ -317,29 +318,46 @@ ColumnUInt8::Ptr HashedDictionary<dictionary_key_type, sparse>::isInHierarchy(
return nullptr;
}
template <DictionaryKeyType dictionary_key_type, bool sparse>
DictionaryHierarchyParentToChildIndexPtr HashedDictionary<dictionary_key_type, sparse>::getHierarchicalIndex() const
{
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{
if (hierarchical_index)
return hierarchical_index;
size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index;
const auto & hierarchical_attribute = attributes[hierarchical_attribute_index];
const CollectionType<UInt64> & parent_keys = std::get<CollectionType<UInt64>>(hierarchical_attribute.container);
HashMap<UInt64, PaddedPODArray<UInt64>> parent_to_child;
parent_to_child.reserve(parent_keys.size());
for (const auto & [key, value] : parent_keys)
parent_to_child[value].emplace_back(key);
return std::make_shared<DictionaryHierarchicalParentToChildIndex>(parent_to_child);
}
else
{
return nullptr;
}
}
template <DictionaryKeyType dictionary_key_type, bool sparse>
ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getDescendants(
ColumnPtr key_column [[maybe_unused]],
const DataTypePtr &,
size_t level [[maybe_unused]]) const
size_t level [[maybe_unused]],
DictionaryHierarchicalParentToChildIndexPtr parent_to_child_index [[maybe_unused]]) const
{
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{
PaddedPODArray<UInt64> keys_backup;
const auto & keys = getColumnVectorData(this, key_column, keys_backup);
size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index;
const auto & hierarchical_attribute = attributes[hierarchical_attribute_index];
const CollectionType<UInt64> & parent_keys = std::get<CollectionType<UInt64>>(hierarchical_attribute.container);
HashMap<UInt64, PaddedPODArray<UInt64>> parent_to_child;
for (const auto & [key, value] : parent_keys)
parent_to_child[value].emplace_back(key);
size_t keys_found;
auto result = getKeysDescendantsArray(keys, parent_to_child, level, keys_found);
auto result = getKeysDescendantsArray(keys, *parent_to_child_index, level, keys_found);
query_count.fetch_add(keys.size(), std::memory_order_relaxed);
found_count.fetch_add(keys_found, std::memory_order_relaxed);
@ -347,7 +365,9 @@ ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getDescendants(
return result;
}
else
{
return nullptr;
}
}
template <DictionaryKeyType dictionary_key_type, bool sparse>
@ -631,6 +651,16 @@ void HashedDictionary<dictionary_key_type, sparse>::loadData()
getFullName());
}
template <DictionaryKeyType dictionary_key_type, bool sparse>
void HashedDictionary<dictionary_key_type, sparse>::buildHierarchyParentToChildIndexIfNeeded()
{
if (!dict_struct.hierarchical_attribute_index)
return;
if (dict_struct.attributes[*dict_struct.hierarchical_attribute_index].bidirectional)
hierarchical_index = getHierarchicalIndex();
}
template <DictionaryKeyType dictionary_key_type, bool sparse>
void HashedDictionary<dictionary_key_type, sparse>::calculateBytesAllocated()
{
@ -684,10 +714,16 @@ void HashedDictionary<dictionary_key_type, sparse>::calculateBytesAllocated()
}
}
bytes_allocated += string_arena.size();
if (update_field_loaded_block)
bytes_allocated += update_field_loaded_block->allocatedBytes();
if (hierarchical_index)
{
hierarchical_index_bytes_allocated = hierarchical_index->getSizeInBytes();
bytes_allocated += hierarchical_index_bytes_allocated;
}
bytes_allocated += string_arena.size();
}
template <DictionaryKeyType dictionary_key_type, bool sparse>

View File

@ -110,10 +110,15 @@ public:
ColumnPtr in_key_column,
const DataTypePtr & key_type) const override;
DictionaryHierarchicalParentToChildIndexPtr getHierarchicalIndex() const override;
size_t getHierarchicalIndexBytesAllocated() const override { return hierarchical_index_bytes_allocated; }
ColumnPtr getDescendants(
ColumnPtr key_column,
const DataTypePtr & key_type,
size_t level) const override;
size_t level,
DictionaryHierarchicalParentToChildIndexPtr parent_to_child_index) const override;
Pipe read(const Names & column_names, size_t max_block_size, size_t num_streams) const override;
@ -194,6 +199,8 @@ private:
void loadData();
void buildHierarchyParentToChildIndexIfNeeded();
void calculateBytesAllocated();
template <typename AttributeType, bool is_nullable, typename ValueSetter, typename DefaultValueExtractor>
@ -218,6 +225,7 @@ private:
std::vector<Attribute> attributes;
size_t bytes_allocated = 0;
size_t hierarchical_index_bytes_allocated = 0;
size_t element_count = 0;
size_t bucket_count = 0;
mutable std::atomic<size_t> query_count{0};
@ -226,6 +234,7 @@ private:
BlockPtr update_field_loaded_block;
Arena string_arena;
NoAttributesCollectionType no_attributes_container;
DictionaryHierarchicalParentToChildIndexPtr hierarchical_index;
};
extern template class HashedDictionary<DictionaryKeyType::Simple, false>;

View File

@ -8,6 +8,22 @@ namespace ErrorCodes
extern const int UNSUPPORTED_METHOD;
}
namespace detail
{
ColumnPtr convertElementsAndOffsetsIntoArray(ElementsAndOffsets && elements_and_offsets)
{
auto elements_column = ColumnVector<UInt64>::create();
elements_column->getData() = std::move(elements_and_offsets.elements);
auto offsets_column = ColumnVector<IColumn::Offset>::create();
offsets_column->getData() = std::move(elements_and_offsets.offsets);
auto column_array = ColumnArray::create(std::move(elements_column), std::move(offsets_column));
return column_array;
}
}
namespace
{
/** In case of cache or direct dictionary we does not have structure with child to parent representation.
@ -84,6 +100,26 @@ namespace
}
}
ColumnPtr getKeysDescendantsArray(
const PaddedPODArray<UInt64> & requested_keys,
const DictionaryHierarchicalParentToChildIndex & parent_to_child_index,
size_t level,
size_t & valid_keys)
{
if (level == 0)
{
detail::GetAllDescendantsStrategy strategy { .level = level };
auto elements_and_offsets = detail::getDescendants(requested_keys, parent_to_child_index, strategy, valid_keys);
return detail::convertElementsAndOffsetsIntoArray(std::move(elements_and_offsets));
}
else
{
detail::GetDescendantsAtSpecificLevelStrategy strategy { .level = level };
auto elements_and_offsets = detail::getDescendants(requested_keys, parent_to_child_index, strategy, valid_keys);
return detail::convertElementsAndOffsetsIntoArray(std::move(elements_and_offsets));
}
}
ColumnPtr getKeysHierarchyDefaultImplementation(
const IDictionary * dictionary,
ColumnPtr key_column,

View File

@ -14,25 +14,65 @@
namespace DB
{
class DictionaryHierarchicalParentToChildIndex;
using DictionaryHierarchyParentToChildIndexPtr = std::shared_ptr<DictionaryHierarchicalParentToChildIndex>;
class DictionaryHierarchicalParentToChildIndex
{
public:
struct KeysRange
{
UInt32 start_index;
UInt32 end_index;
};
explicit DictionaryHierarchicalParentToChildIndex(const HashMap<UInt64, PaddedPODArray<UInt64>> & parent_to_children_map_)
{
size_t parent_to_children_map_size = parent_to_children_map_.size();
keys.reserve(parent_to_children_map_size);
parent_to_children_keys_range.reserve(parent_to_children_map_size);
for (auto & [parent, children] : parent_to_children_map_)
{
size_t keys_size = keys.size();
UInt32 start_index = static_cast<UInt32>(keys_size);
UInt32 end_index = start_index + static_cast<UInt32>(children.size());
keys.insert(children.begin(), children.end());
parent_to_children_keys_range[parent] = KeysRange{start_index, end_index};
}
}
size_t getSizeInBytes() const
{
return parent_to_children_keys_range.getBufferSizeInBytes() + (keys.size() * sizeof(UInt64));
}
/// Map parent key to range of children from keys array
HashMap<UInt64, KeysRange> parent_to_children_keys_range;
/// Array of keys in hierarchy
PaddedPODArray<UInt64> keys;
};
namespace detail
{
template <typename KeyType>
struct ElementsAndOffsets
{
PaddedPODArray<KeyType> elements;
PaddedPODArray<UInt64> elements;
PaddedPODArray<IColumn::Offset> offsets;
};
template <typename T>
struct IsKeyValidFuncInterface
{
bool operator()(T key [[maybe_unused]]) { return false; }
bool operator()(UInt64 key [[maybe_unused]]) { return false; }
};
template <typename T>
struct GetParentKeyFuncInterface
{
std::optional<T> operator()(T key [[maybe_unused]]) { return {}; }
std::optional<UInt64> operator()(UInt64 key [[maybe_unused]]) { return {}; }
};
/** Calculate hierarchy for keys iterating the hierarchy from child to parent using get_parent_key_func provided by client.
@ -54,16 +94,16 @@ namespace detail
* Elements: [1, 2, 1, 3, 1, 4, 2, 1]
* Offsets: [1, 3, 5, 8, 8]
*/
template <typename KeyType, typename IsKeyValidFunc, typename GetParentKeyFunc>
ElementsAndOffsets<KeyType> getHierarchy(
const PaddedPODArray<KeyType> & keys,
const KeyType & hierarchy_null_value,
template <typename IsKeyValidFunc, typename GetParentKeyFunc>
ElementsAndOffsets getHierarchy(
const PaddedPODArray<UInt64> & keys,
const UInt64 & hierarchy_null_value,
IsKeyValidFunc && is_key_valid_func,
GetParentKeyFunc && get_parent_key_func)
{
size_t hierarchy_keys_size = keys.size();
PaddedPODArray<KeyType> elements;
PaddedPODArray<UInt64> elements;
elements.reserve(hierarchy_keys_size);
PaddedPODArray<IColumn::Offset> offsets;
@ -75,7 +115,7 @@ namespace detail
size_t array_element_offset;
};
HashMap<KeyType, OffsetInArray> already_processes_keys_to_offset;
HashMap<UInt64, OffsetInArray> already_processes_keys_to_offset;
already_processes_keys_to_offset.reserve(hierarchy_keys_size);
for (size_t i = 0; i < hierarchy_keys_size; ++i)
@ -123,7 +163,7 @@ namespace detail
elements.emplace_back(hierarchy_key);
++current_hierarchy_depth;
std::optional<KeyType> parent_key = std::forward<GetParentKeyFunc>(get_parent_key_func)(hierarchy_key);
std::optional<UInt64> parent_key = std::forward<GetParentKeyFunc>(get_parent_key_func)(hierarchy_key);
if (!parent_key.has_value())
break;
@ -134,7 +174,7 @@ namespace detail
offsets.emplace_back(elements.size());
}
ElementsAndOffsets<KeyType> result = {std::move(elements), std::move(offsets)};
ElementsAndOffsets result = {std::move(elements), std::move(offsets)};
return result;
}
@ -146,11 +186,11 @@ namespace detail
*
* Not: keys size must be equal to in_keys_size.
*/
template <typename KeyType, typename IsKeyValidFunc, typename GetParentKeyFunc>
template <typename IsKeyValidFunc, typename GetParentKeyFunc>
PaddedPODArray<UInt8> getIsInHierarchy(
const PaddedPODArray<KeyType> & keys,
const PaddedPODArray<KeyType> & in_keys,
const KeyType & hierarchy_null_value,
const PaddedPODArray<UInt64> & keys,
const PaddedPODArray<UInt64> & in_keys,
const UInt64 & hierarchy_null_value,
IsKeyValidFunc && is_key_valid_func,
GetParentKeyFunc && get_parent_func)
{
@ -159,7 +199,7 @@ namespace detail
PaddedPODArray<UInt8> result;
result.resize_fill(keys.size());
detail::ElementsAndOffsets<KeyType> hierarchy = detail::getHierarchy(
detail::ElementsAndOffsets hierarchy = detail::getHierarchy(
keys,
hierarchy_null_value,
std::forward<IsKeyValidFunc>(is_key_valid_func),
@ -216,19 +256,22 @@ namespace detail
* Result: [1], [2, 3], [4], [], [];
* Offsets: [1, 3, 4, 4, 4];
*/
template <typename KeyType, typename Strategy>
ElementsAndOffsets<KeyType> getDescendants(
const PaddedPODArray<KeyType> & keys,
const HashMap<KeyType, PaddedPODArray<KeyType>> & parent_to_child,
template <typename Strategy>
ElementsAndOffsets getDescendants(
const PaddedPODArray<UInt64> & keys,
const DictionaryHierarchicalParentToChildIndex & parent_to_child_index,
Strategy strategy,
size_t & valid_keys)
{
auto & parent_to_children_keys_range = parent_to_child_index.parent_to_children_keys_range;
auto & children_keys = parent_to_child_index.keys;
/// If strategy is GetAllDescendantsStrategy we try to cache and later reuse previously calculated descendants.
/// If strategy is GetDescendantsAtSpecificLevelStrategy we does not use cache strategy.
size_t keys_size = keys.size();
valid_keys = 0;
PaddedPODArray<KeyType> descendants;
PaddedPODArray<UInt64> descendants;
descendants.reserve(keys_size);
PaddedPODArray<IColumn::Offset> descendants_offsets;
@ -241,18 +284,18 @@ namespace detail
};
static constexpr Int64 key_range_requires_update = -1;
HashMap<KeyType, Range> already_processed_keys_to_range [[maybe_unused]];
HashMap<UInt64, Range> already_processed_keys_to_range [[maybe_unused]];
if constexpr (std::is_same_v<Strategy, GetAllDescendantsStrategy>)
already_processed_keys_to_range.reserve(keys_size);
struct KeyAndDepth
{
KeyType key;
UInt64 key;
Int64 depth;
};
HashSet<KeyType> already_processed_keys_during_loop;
HashSet<UInt64> already_processed_keys_during_loop;
already_processed_keys_during_loop.reserve(keys_size);
PaddedPODArray<KeyAndDepth> next_keys_to_process_stack;
@ -262,9 +305,9 @@ namespace detail
for (size_t i = 0; i < keys_size; ++i)
{
const KeyType & requested_key = keys[i];
const UInt64 & requested_key = keys[i];
if (parent_to_child.find(requested_key) == nullptr)
if (parent_to_children_keys_range.find(requested_key) == nullptr)
{
descendants_offsets.emplace_back(descendants.size());
continue;
@ -282,7 +325,7 @@ namespace detail
{
KeyAndDepth key_to_process = next_keys_to_process_stack.back();
KeyType key = key_to_process.key;
UInt64 key = key_to_process.key;
Int64 depth = key_to_process.depth;
next_keys_to_process_stack.pop_back();
@ -329,7 +372,7 @@ namespace detail
}
}
const auto * it = parent_to_child.find(key);
const auto * it = parent_to_children_keys_range.find(key);
if (!it || depth >= DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH)
continue;
@ -352,15 +395,26 @@ namespace detail
++depth;
const auto & children = it->getMapped();
DictionaryHierarchicalParentToChildIndex::KeysRange children_range = it->getMapped();
for (auto child_key : children)
for (; children_range.start_index < children_range.end_index; ++children_range.start_index)
{
auto child_key = children_keys[children_range.start_index];
/// In case of GetAllDescendantsStrategy we add any descendant to result array
/// If strategy is GetDescendantsAtSpecificLevelStrategy we require depth == level
if (std::is_same_v<Strategy, GetAllDescendantsStrategy> || depth == level)
if constexpr (std::is_same_v<Strategy, GetAllDescendantsStrategy>)
descendants.emplace_back(child_key);
if constexpr (std::is_same_v<Strategy, GetDescendantsAtSpecificLevelStrategy>)
{
if (depth == level)
{
descendants.emplace_back(child_key);
continue;
}
}
next_keys_to_process_stack.emplace_back(KeyAndDepth{child_key, depth});
}
}
@ -370,24 +424,12 @@ namespace detail
descendants_offsets.emplace_back(descendants.size());
}
ElementsAndOffsets<KeyType> result = {std::move(descendants), std::move(descendants_offsets)};
ElementsAndOffsets result = {std::move(descendants), std::move(descendants_offsets)};
return result;
}
/// Converts ElementAndOffsets structure into ArrayColumn
template<typename KeyType>
ColumnPtr convertElementsAndOffsetsIntoArray(ElementsAndOffsets<KeyType> && elements_and_offsets)
{
auto elements_column = ColumnVector<KeyType>::create();
elements_column->getData() = std::move(elements_and_offsets.elements);
auto offsets_column = ColumnVector<IColumn::Offset>::create();
offsets_column->getData() = std::move(elements_and_offsets.offsets);
auto column_array = ColumnArray::create(std::move(elements_column), std::move(offsets_column));
return column_array;
}
ColumnPtr convertElementsAndOffsetsIntoArray(ElementsAndOffsets && elements_and_offsets);
}
/// Returns hierarchy array column for keys
@ -432,26 +474,11 @@ ColumnUInt8::Ptr getKeysIsInHierarchyColumn(
/// Returns descendants array column for keys
///
/// @param valid_keys - number of keys that are valid in parent_to_child map
template <typename KeyType>
ColumnPtr getKeysDescendantsArray(
const PaddedPODArray<KeyType> & requested_keys,
const HashMap<KeyType, PaddedPODArray<KeyType>> & parent_to_child,
const PaddedPODArray<UInt64> & requested_keys,
const DictionaryHierarchicalParentToChildIndex & parent_to_child_index,
size_t level,
size_t & valid_keys)
{
if (level == 0)
{
detail::GetAllDescendantsStrategy strategy { .level = level };
auto elements_and_offsets = detail::getDescendants(requested_keys, parent_to_child, strategy, valid_keys);
return detail::convertElementsAndOffsetsIntoArray(std::move(elements_and_offsets));
}
else
{
detail::GetDescendantsAtSpecificLevelStrategy strategy { .level = level };
auto elements_and_offsets = detail::getDescendants(requested_keys, parent_to_child, strategy, valid_keys);
return detail::convertElementsAndOffsetsIntoArray(std::move(elements_and_offsets));
}
}
size_t & valid_keys);
/** Default getHierarchy implementation for dictionaries that does not have structure with child to parent representation.
* Implementation will build such structure with getColumn calls, and then getHierarchy for such structure.

View File

@ -24,6 +24,9 @@ namespace ErrorCodes
class IDictionary;
using DictionaryPtr = std::unique_ptr<IDictionary>;
class DictionaryHierarchicalParentToChildIndex;
using DictionaryHierarchicalParentToChildIndexPtr = std::shared_ptr<DictionaryHierarchicalParentToChildIndex>;
/** DictionaryKeyType provides IDictionary client information about
* which key type is supported by dictionary.
*
@ -228,10 +231,23 @@ public:
getDictionaryID().getNameForLogs());
}
virtual DictionaryHierarchicalParentToChildIndexPtr getHierarchicalIndex() const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"Method getHierarchicalIndex is not supported for {} dictionary.",
getDictionaryID().getNameForLogs());
}
virtual size_t getHierarchicalIndexBytesAllocated() const
{
return 0;
}
virtual ColumnPtr getDescendants(
ColumnPtr key_column [[maybe_unused]],
const DataTypePtr & key_type [[maybe_unused]],
size_t level [[maybe_unused]]) const
size_t level [[maybe_unused]],
DictionaryHierarchicalParentToChildIndexPtr parent_to_child_index [[maybe_unused]]) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"Method getDescendants is not supported for {} dictionary.",

View File

@ -290,6 +290,14 @@ void buildSingleAttribute(
attribute_element->appendChild(hierarchical_element);
}
if (dict_attr->bidirectional)
{
AutoPtr<Element> bidirectional_element(doc->createElement("bidirectional"));
AutoPtr<Text> bidirectional(doc->createTextNode("true"));
bidirectional_element->appendChild(bidirectional);
attribute_element->appendChild(bidirectional_element);
}
if (dict_attr->injective)
{
AutoPtr<Element> injective_element(doc->createElement("injective"));

View File

@ -151,13 +151,15 @@ TEST(HierarchyDictionariesUtils, getDescendants)
parent_to_child[1].emplace_back(3);
parent_to_child[2].emplace_back(4);
auto parent_to_child_index = std::make_shared<DictionaryHierarchicalParentToChildIndex>(parent_to_child);
PaddedPODArray<UInt64> keys = {0, 1, 2, 3, 4};
{
size_t keys_found;
auto result = DB::detail::getDescendants(
keys,
parent_to_child,
*parent_to_child_index,
DB::detail::GetAllDescendantsStrategy(),
keys_found);
@ -175,7 +177,7 @@ TEST(HierarchyDictionariesUtils, getDescendants)
size_t keys_found;
auto result = DB::detail::getDescendants(
keys,
parent_to_child,
*parent_to_child_index,
DB::detail::GetDescendantsAtSpecificLevelStrategy{1},
keys_found);
@ -195,13 +197,15 @@ TEST(HierarchyDictionariesUtils, getDescendants)
parent_to_child[1].emplace_back(2);
parent_to_child[2].emplace_back(1);
auto parent_to_child_index = std::make_shared<DictionaryHierarchicalParentToChildIndex>(parent_to_child);
PaddedPODArray<UInt64> keys = {1, 2, 3};
{
size_t keys_found;
auto result = DB::detail::getDescendants(
keys,
parent_to_child,
*parent_to_child_index,
DB::detail::GetAllDescendantsStrategy(),
keys_found);
@ -219,7 +223,7 @@ TEST(HierarchyDictionariesUtils, getDescendants)
size_t keys_found;
auto result = DB::detail::getDescendants(
keys,
parent_to_child,
*parent_to_child_index,
DB::detail::GetDescendantsAtSpecificLevelStrategy{1},
keys_found);

View File

@ -83,6 +83,9 @@ public:
UInt32 getRefCount(const String & path) const override { return delegate->getRefCount(path); }
void syncRevision(UInt64 revision) override { delegate->syncRevision(revision); }
UInt64 getRevision() const override { return delegate->getRevision(); }
protected:
Executor & getExecutor() override;

View File

@ -343,6 +343,14 @@ public:
/// other alive harlinks will not be removed.
virtual UInt32 getRefCount(const String &) const { return 0; }
/// Revision is an incremental counter of disk operation.
/// Revision currently exisis only in DiskS3.
/// It is used to save current state during backup and restore that state from backup.
/// This method sets current disk revision if it lower than required.
virtual void syncRevision(UInt64) {}
/// Return current disk revision.
virtual UInt64 getRevision() const { return 0; }
protected:
friend class DiskDecorator;

View File

@ -114,13 +114,15 @@ DiskS3::DiskS3(
FileCachePtr cache_,
ContextPtr context_,
SettingsPtr settings_,
GetDiskSettings settings_getter_)
GetDiskSettings settings_getter_,
String operation_log_suffix_)
: IDiskRemote(name_, s3_root_path_, metadata_disk_, std::move(cache_), "DiskS3", settings_->thread_pool_size)
, bucket(std::move(bucket_))
, version_id(std::move(version_id_))
, current_settings(std::move(settings_))
, settings_getter(settings_getter_)
, context(context_)
, operation_log_suffix(operation_log_suffix_)
{
}
@ -289,7 +291,7 @@ void DiskS3::shutdown()
void DiskS3::createFileOperationObject(const String & operation_name, UInt64 revision, const DiskS3::ObjectMetadata & metadata)
{
auto settings = current_settings.get();
const String key = "operations/r" + revisionToString(revision) + "-" + operation_name;
const String key = "operations/r" + revisionToString(revision) + operation_log_suffix + "-" + operation_name;
WriteBufferFromS3 buffer(
settings->client,
bucket,
@ -862,6 +864,36 @@ void DiskS3::processRestoreFiles(const String & source_bucket, const String & so
}
}
void DiskS3::moveRecursiveOrRemove(const String & from_path, const String & to_path, bool send_metadata)
{
if (exists(to_path))
{
if (send_metadata)
{
auto revision = ++revision_counter;
const ObjectMetadata object_metadata {
{"from_path", from_path},
{"to_path", to_path}
};
createFileOperationObject("rename", revision, object_metadata);
}
if (isDirectory(from_path))
{
for (auto it = iterateDirectory(from_path); it->isValid(); it->next())
moveRecursiveOrRemove(it->path(), fs::path(to_path) / it->name(), false);
}
else
{
removeFile(from_path);
LOG_WARNING(log, "Collision in S3 operation log: rename from '{}' to '{}', file removed", from_path, to_path);
}
}
else
{
moveFile(from_path, to_path, send_metadata);
}
}
void DiskS3::restoreFileOperations(const RestoreInformation & restore_information)
{
auto settings = current_settings.get();
@ -904,7 +936,7 @@ void DiskS3::restoreFileOperations(const RestoreInformation & restore_informatio
auto to_path = object_metadata["to_path"];
if (exists(from_path))
{
moveFile(from_path, to_path, send_metadata);
moveRecursiveOrRemove(from_path, to_path, send_metadata);
LOG_TRACE(log, "Revision {}. Restored rename {} -> {}", revision, from_path, to_path);
if (restore_information.detached && isDirectory(to_path))
@ -987,9 +1019,10 @@ void DiskS3::restoreFileOperations(const RestoreInformation & restore_informatio
std::tuple<UInt64, String> DiskS3::extractRevisionAndOperationFromKey(const String & key)
{
String revision_str;
String suffix;
String operation;
re2::RE2::FullMatch(key, key_regexp, &revision_str, &operation);
re2::RE2::FullMatch(key, key_regexp, &revision_str, &suffix, &operation);
return {(revision_str.empty() ? UNKNOWN_REVISION : static_cast<UInt64>(std::bitset<64>(revision_str).to_ullong())), operation};
}
@ -1032,6 +1065,17 @@ void DiskS3::applyNewSettings(const Poco::Util::AbstractConfiguration & config,
exec->setMaxThreads(current_settings.get()->thread_pool_size);
}
void DiskS3::syncRevision(UInt64 revision)
{
UInt64 local_revision = revision_counter.load();
while ((revision > local_revision) && revision_counter.compare_exchange_weak(local_revision, revision));
}
UInt64 DiskS3::getRevision() const
{
return revision_counter.load();
}
DiskS3Settings::DiskS3Settings(
const std::shared_ptr<Aws::S3::S3Client> & client_,
const S3Settings::ReadWriteSettings & s3_settings_,

View File

@ -71,7 +71,8 @@ public:
FileCachePtr cache_,
ContextPtr context_,
SettingsPtr settings_,
GetDiskSettings settings_getter_);
GetDiskSettings settings_getter_,
String operation_log_suffix_);
std::unique_ptr<ReadBufferFromFileBase> readFile(
const String & path,
@ -114,6 +115,9 @@ public:
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String &, const DisksMap &) override;
void syncRevision(UInt64 revision) override;
UInt64 getRevision() const override;
private:
void createFileOperationObject(const String & operation_name, UInt64 revision, const ObjectMetadata & metadata);
/// Converts revision to binary string with leading zeroes (64 bit).
@ -156,6 +160,10 @@ private:
/// Forms detached path '../../detached/part_name/' from '../../part_name/'
static String pathToDetached(const String & source_path);
/// Move file or files in directory when possible and remove files in other case
/// to restore by S3 operation log with same operations from different replicas
void moveRecursiveOrRemove(const String & from_path, const String & to_path, bool send_metadata);
const String bucket;
const String version_id;
@ -171,8 +179,8 @@ private:
/// File at path {metadata_path}/restore contains metadata restore information
inline static const String RESTORE_FILE_NAME = "restore";
/// Key has format: ../../r{revision}-{operation}
const re2::RE2 key_regexp {".*/r(\\d+)-(\\w+)$"};
/// Key has format: ../../r{revision}(-{hostname})-{operation}
const re2::RE2 key_regexp {".*/r(\\d+)(-[\\w\\d\\-\\.]+)?-(\\w+)$"};
/// Object contains information about schema version.
inline static const String SCHEMA_VERSION_OBJECT = ".SCHEMA_VERSION";
@ -182,6 +190,8 @@ private:
const std::vector<String> data_roots {"data", "store"};
ContextPtr context;
String operation_log_suffix;
};
}

View File

@ -9,6 +9,7 @@
#if USE_AWS_S3
#include <aws/core/client/DefaultRetryStrategy.h>
#include <base/getFQDNOrHostName.h>
#include <IO/S3Common.h>
#include "DiskS3.h"
#include "Disks/DiskCacheWrapper.h"
@ -200,7 +201,8 @@ void registerDiskS3(DiskFactory & factory)
std::move(cache),
context,
getSettings(config, config_prefix, context),
getSettings);
getSettings,
"-" + getFQDNOrHostName());
/// This code is used only to check access to the corresponding disk.
if (!config.getBool(config_prefix + ".skip_access_check", false))

View File

@ -98,6 +98,7 @@ protected:
}
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForNothing() const override { return false; }
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
private:

View File

@ -2516,6 +2516,8 @@ protected:
}
bool useDefaultImplementationForNulls() const override { return false; }
/// CAST(Nothing, T) -> T
bool useDefaultImplementationForNothing() const override { return false; }
bool useDefaultImplementationForConstants() const override { return true; }
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }

View File

@ -24,8 +24,8 @@ void registerFunctionsExternalDictionaries(FunctionFactory & factory)
factory.registerFunction<FunctionDictGetString>();
factory.registerFunction<FunctionDictGetHierarchy>();
factory.registerFunction<FunctionDictIsIn>();
factory.registerFunction<FunctionDictGetChildren>();
factory.registerFunction<FunctionDictGetDescendants>();
factory.registerFunction<FunctionDictGetChildrenOverloadResolver>();
factory.registerFunction<FunctionDictGetDescendantsOverloadResolver>();
factory.registerFunction<FunctionDictGetUInt8OrDefault>();
factory.registerFunction<FunctionDictGetUInt16OrDefault>();
factory.registerFunction<FunctionDictGetUInt32OrDefault>();

View File

@ -1053,86 +1053,159 @@ private:
mutable FunctionDictHelper helper;
};
class FunctionDictGetChildren final : public IFunction
class FunctionDictGetDescendantsExecutable final : public IExecutableFunction
{
public:
static constexpr auto name = "dictGetChildren";
static FunctionPtr create(ContextPtr context)
{
return std::make_shared<FunctionDictGetChildren>(context);
}
explicit FunctionDictGetChildren(ContextPtr context_)
: helper(context_) {}
FunctionDictGetDescendantsExecutable(
String name_,
size_t level_,
DictionaryHierarchicalParentToChildIndexPtr hierarchical_parent_to_child_index,
std::shared_ptr<FunctionDictHelper> dictionary_helper_)
: name(std::move(name_))
, level(level_)
, hierarchical_parent_to_child_index(std::move(hierarchical_parent_to_child_index))
, dictionary_helper(std::move(dictionary_helper_))
{}
String getName() const override { return name; }
private:
size_t getNumberOfArguments() const override { return 2; }
bool useDefaultImplementationForConstants() const override { return true; }
bool useDefaultImplementationForConstants() const final { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; }
bool isDeterministic() const override { return false; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (!isString(arguments[0].type))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of first argument of function {}. Expected String. Actual type {}",
getName(),
arguments[0].type->getName());
auto dictionary = helper.getDictionary(arguments[0].column);
const auto & hierarchical_attribute = helper.getDictionaryHierarchicalAttribute(dictionary);
return std::make_shared<DataTypeArray>(hierarchical_attribute.type);
}
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 2}; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
if (input_rows_count == 0)
return result_type->createColumn();
auto dictionary = helper.getDictionary(arguments[0].column);
const auto & hierarchical_attribute = helper.getDictionaryHierarchicalAttribute(dictionary);
auto dictionary = dictionary_helper->getDictionary(arguments[0].column);
const auto & hierarchical_attribute = dictionary_helper->getDictionaryHierarchicalAttribute(dictionary);
auto key_column = ColumnWithTypeAndName{arguments[1].column->convertToFullColumnIfConst(), arguments[1].type, arguments[1].name};
auto key_column_casted = castColumnAccurate(key_column, hierarchical_attribute.type);
ColumnPtr result = dictionary->getDescendants(key_column_casted, hierarchical_attribute.type, 1);
ColumnPtr result = dictionary->getDescendants(key_column_casted, hierarchical_attribute.type, level, hierarchical_parent_to_child_index);
return result;
}
mutable FunctionDictHelper helper;
String name;
size_t level;
DictionaryHierarchicalParentToChildIndexPtr hierarchical_parent_to_child_index;
std::shared_ptr<FunctionDictHelper> dictionary_helper;
};
class FunctionDictGetDescendants final : public IFunction
class FunctionDictGetDescendantsBase final : public IFunctionBase
{
public:
static constexpr auto name = "dictGetDescendants";
static FunctionPtr create(ContextPtr context)
{
return std::make_shared<FunctionDictGetDescendants>(context);
}
explicit FunctionDictGetDescendants(ContextPtr context_)
: helper(context_) {}
FunctionDictGetDescendantsBase(
String name_,
const DataTypes & argument_types_,
const DataTypePtr & result_type_,
size_t level_,
DictionaryHierarchicalParentToChildIndexPtr hierarchical_parent_to_child_index,
std::shared_ptr<FunctionDictHelper> helper_)
: name(std::move(name_))
, argument_types(argument_types_)
, result_type(result_type_)
, level(level_)
, hierarchical_parent_to_child_index(std::move(hierarchical_parent_to_child_index))
, helper(std::move(helper_))
{}
String getName() const override { return name; }
private:
size_t getNumberOfArguments() const override { return 0; }
bool isVariadic() const override { return true; }
const DataTypes & getArgumentTypes() const override { return argument_types; }
const DataTypePtr & getResultType() const override { return result_type; }
bool useDefaultImplementationForConstants() const final { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 2}; }
bool isDeterministic() const override { return false; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName &) const override
{
return std::make_shared<FunctionDictGetDescendantsExecutable>(name, level, hierarchical_parent_to_child_index, helper);
}
String name;
DataTypes argument_types;
DataTypePtr result_type;
size_t level;
DictionaryHierarchicalParentToChildIndexPtr hierarchical_parent_to_child_index;
std::shared_ptr<FunctionDictHelper> helper;
};
struct FunctionDictGetDescendantsStrategy
{
static constexpr auto name = "dictGetDescendants";
static constexpr size_t default_level = 0;
static constexpr size_t number_of_arguments = 0;
static constexpr bool is_variadic = true;
};
struct FunctionDictGetChildrenStrategy
{
static constexpr auto name = "dictGetChildren";
static constexpr size_t default_level = 1;
static constexpr size_t number_of_arguments = 2;
static constexpr bool is_variadic = false;
};
template <typename Strategy>
class FunctionDictGetDescendantsOverloadResolverImpl final : public IFunctionOverloadResolver
{
public:
static constexpr auto name = Strategy::name;
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return Strategy::number_of_arguments; }
bool isVariadic() const override { return Strategy::is_variadic; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 2}; }
bool isDeterministic() const override { return false; }
explicit FunctionDictGetDescendantsOverloadResolverImpl(ContextPtr context)
: dictionary_helper(std::make_shared<FunctionDictHelper>(std::move(context)))
{}
static FunctionOverloadResolverPtr create(ContextPtr context)
{
return std::make_shared<FunctionDictGetDescendantsOverloadResolverImpl>(std::move(context));
}
FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const override
{
auto dictionary = dictionary_helper->getDictionary(arguments[0].column);
auto hierarchical_parent_to_child_index = dictionary->getHierarchicalIndex();
size_t level = Strategy::default_level;
if (arguments.size() == 3)
{
if (!arguments[2].column || !isColumnConst(*arguments[2].column))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of third argument of function {}. Expected const unsigned integer.",
getName());
auto value = static_cast<Int64>(arguments[2].column->getInt(0));
if (value < 0)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of third argument of function {}. Expected const unsigned integer.",
getName());
level = static_cast<size_t>(value);
}
DataTypes argument_types;
argument_types.reserve(arguments.size());
for (const auto & argument : arguments)
argument_types.emplace_back(argument.type);
return std::make_shared<FunctionDictGetDescendantsBase>(name, argument_types, result_type, level, hierarchical_parent_to_child_index, dictionary_helper);
}
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
size_t arguments_size = arguments.size();
@ -1158,47 +1231,16 @@ private:
arguments[2].type->getName());
}
auto dictionary = helper.getDictionary(arguments[0].column);
const auto & hierarchical_attribute = helper.getDictionaryHierarchicalAttribute(dictionary);
auto dictionary = dictionary_helper->getDictionary(arguments[0].column);
const auto & hierarchical_attribute = dictionary_helper->getDictionaryHierarchicalAttribute(dictionary);
return std::make_shared<DataTypeArray>(hierarchical_attribute.type);
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
if (input_rows_count == 0)
return result_type->createColumn();
auto dictionary = helper.getDictionary(arguments[0].column);
const auto & hierarchical_attribute = helper.getDictionaryHierarchicalAttribute(dictionary);
size_t level = 0;
if (arguments.size() == 3)
{
if (!isColumnConst(*arguments[2].column))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of third argument of function {}. Expected const unsigned integer.",
getName());
auto value = static_cast<Int64>(arguments[2].column->getInt(0));
if (value < 0)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of third argument of function {}. Expected const unsigned integer.",
getName());
level = static_cast<size_t>(value);
}
auto key_column = ColumnWithTypeAndName{arguments[1].column->convertToFullColumnIfConst(), arguments[1].type, arguments[1].name};
auto key_column_casted = castColumnAccurate(key_column, hierarchical_attribute.type);
ColumnPtr result = dictionary->getDescendants(key_column_casted, hierarchical_attribute.type, level);
return result;
}
mutable FunctionDictHelper helper;
std::shared_ptr<FunctionDictHelper> dictionary_helper;
};
using FunctionDictGetDescendantsOverloadResolver = FunctionDictGetDescendantsOverloadResolverImpl<FunctionDictGetDescendantsStrategy>;
using FunctionDictGetChildrenOverloadResolver = FunctionDictGetDescendantsOverloadResolverImpl<FunctionDictGetChildrenStrategy>;
}

View File

@ -50,7 +50,11 @@ public:
return expr_columns.getByName(signature->return_name).column;
}
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForNulls() const override { return false; }
/// It's possible if expression_actions contains function that don't use
/// default implementation for Nothing.
/// Example: arrayMap(x -> CAST(x, 'UInt8'), []);
bool useDefaultImplementationForNothing() const override { return false; }
private:
ExpressionActionsPtr expression_actions;
@ -118,6 +122,10 @@ public:
String getName() const override { return "FunctionCapture"; }
bool useDefaultImplementationForNulls() const override { return false; }
/// It's possible if expression_actions contains function that don't use
/// default implementation for Nothing and one of captured columns can be Nothing
/// Example: SELECT arrayMap(x -> [x, arrayElement(y, 0)], []), [] as y
bool useDefaultImplementationForNothing() const override { return false; }
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
@ -247,6 +255,8 @@ public:
String getName() const override { return name; }
bool useDefaultImplementationForNulls() const override { return false; }
/// See comment in ExecutableFunctionCapture.
bool useDefaultImplementationForNothing() const override { return false; }
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName &) const override { return return_type; }
size_t getNumberOfArguments() const override { return capture->captured_types.size(); }

View File

@ -4,18 +4,11 @@
#include "Slices.h"
#include "sliceEqualElements.h"
#if defined(__SSE4_2__)
#include <emmintrin.h>
#include <smmintrin.h>
#include <nmmintrin.h>
#endif
#if defined(__AVX2__)
#include <immintrin.h>
#endif
#include <Common/TargetSpecific.h>
#if USE_MULTITARGET_CODE
#include <immintrin.h>
#endif
namespace DB::GatherUtils
{
@ -67,15 +60,14 @@ inline ALWAYS_INLINE bool hasAllIntegralLoopRemainder(
return true;
}
#if defined(__AVX2__)
#if USE_MULTITARGET_CODE
DECLARE_AVX2_SPECIFIC_CODE (
// AVX2 Int64, UInt64 specialization
template<typename IntType>
requires (std::is_same_v<IntType, Int64> || std::is_same_v<IntType, UInt64>)
inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt64(
bool sliceHasImplAnyAllImplInt64(
const NumericArraySlice<IntType> & first,
const NumericArraySlice<IntType> & second,
const UInt8 * first_null_map,
@ -165,7 +157,7 @@ inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt64(
// AVX2 Int32, UInt32 specialization
template<typename IntType>
requires (std::is_same_v<IntType, Int32> || std::is_same_v<IntType, UInt32>)
inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt32(
bool sliceHasImplAnyAllImplInt32(
const NumericArraySlice<IntType> & first,
const NumericArraySlice<IntType> & second,
const UInt8 * first_null_map,
@ -278,7 +270,7 @@ inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt32(
// AVX2 Int16, UInt16 specialization
template<typename IntType>
requires (std::is_same_v<IntType, Int16> || std::is_same_v<IntType, UInt16>)
inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt16(
bool sliceHasImplAnyAllImplInt16(
const NumericArraySlice<IntType> & first,
const NumericArraySlice<IntType> & second,
const UInt8 * first_null_map,
@ -422,10 +414,6 @@ inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt16(
)
#endif
#if defined(__SSE4_2__)
DECLARE_SSE42_SPECIFIC_CODE (
// SSE4.2 Int64, UInt64 specialization
@ -895,8 +883,6 @@ inline ALWAYS_INLINE bool sliceHasImplAnyAll(const FirstSliceType & first, const
#if USE_MULTITARGET_CODE
if constexpr (search_type == ArraySearchType::All && std::is_same_v<FirstSliceType, SecondSliceType>)
{
#if defined(__AVX2__)
if (isArchSupported(TargetArch::AVX2))
{
if constexpr (std::is_same_v<FirstSliceType, NumericArraySlice<Int16>> || std::is_same_v<FirstSliceType, NumericArraySlice<UInt16>>)
@ -912,13 +898,12 @@ inline ALWAYS_INLINE bool sliceHasImplAnyAll(const FirstSliceType & first, const
return GatherUtils::TargetSpecific::AVX2::sliceHasImplAnyAllImplInt64(first, second, first_null_map, second_null_map);
}
}
#endif
if (isArchSupported(TargetArch::SSE42))
{
if constexpr (std::is_same_v<FirstSliceType, NumericArraySlice<Int8>> || std::is_same_v<FirstSliceType, NumericArraySlice<UInt8>>)
{
return TargetSpecific::SSE42::sliceHasImplAnyAllImplInt8(first, second, first_null_map, second_null_map);
return GatherUtils::TargetSpecific::SSE42::sliceHasImplAnyAllImplInt8(first, second, first_null_map, second_null_map);
}
else if constexpr (std::is_same_v<FirstSliceType, NumericArraySlice<Int16>> || std::is_same_v<FirstSliceType, NumericArraySlice<UInt16>>)
{

View File

@ -9,6 +9,7 @@
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnLowCardinality.h>
#include <Columns/ColumnSparse.h>
#include <Columns/ColumnNothing.h>
#include <DataTypes/DataTypeNothing.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/Native.h>
@ -203,6 +204,32 @@ ColumnPtr IExecutableFunction::defaultImplementationForNulls(
return nullptr;
}
ColumnPtr IExecutableFunction::defaultImplementationForNothing(
const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count) const
{
if (!useDefaultImplementationForNothing())
return nullptr;
bool is_nothing_type_presented = false;
for (const auto & arg : args)
is_nothing_type_presented |= isNothing(arg.type);
if (!is_nothing_type_presented)
return nullptr;
if (!isNothing(result_type))
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Function {} with argument with type Nothing and default implementation for Nothing "
"is expected to return result with type Nothing, got {}",
getName(),
result_type->getName());
if (input_rows_count > 0)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot create non-empty column with type Nothing");
return ColumnNothing::create(0);
}
ColumnPtr IExecutableFunction::executeWithoutLowCardinalityColumns(
const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const
{
@ -212,6 +239,9 @@ ColumnPtr IExecutableFunction::executeWithoutLowCardinalityColumns(
if (auto res = defaultImplementationForNulls(args, result_type, input_rows_count, dry_run))
return res;
if (auto res = defaultImplementationForNothing(args, result_type, input_rows_count))
return res;
ColumnPtr res;
if (dry_run)
res = executeDryRunImpl(args, result_type, input_rows_count);
@ -436,6 +466,15 @@ DataTypePtr IFunctionOverloadResolver::getReturnTypeWithoutLowCardinality(const
}
}
if (!arguments.empty() && useDefaultImplementationForNothing())
{
for (const auto & arg : arguments)
{
if (isNothing(arg.type))
return std::make_shared<DataTypeNothing>();
}
}
return getReturnTypeImpl(arguments);
}

View File

@ -63,6 +63,11 @@ protected:
*/
virtual bool useDefaultImplementationForNulls() const { return true; }
/** Default implementation in presence of arguments with type Nothing is the following:
* If some of arguments have type Nothing then default implementation is to return constant column with type Nothing
*/
virtual bool useDefaultImplementationForNothing() const { return true; }
/** If the function have non-zero number of arguments,
* and if all arguments are constant, that we could automatically provide default implementation:
* arguments are converted to ordinary columns with single value, then function is executed as usual,
@ -100,6 +105,9 @@ private:
ColumnPtr defaultImplementationForNulls(
const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const;
ColumnPtr defaultImplementationForNothing(
const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count) const;
ColumnPtr executeWithoutLowCardinalityColumns(
const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const;
@ -166,8 +174,8 @@ public:
/** If function isSuitableForConstantFolding then, this method will be called during query analyzis
* if some arguments are constants. For example logical functions (AndFunction, OrFunction) can
* return they result based on some constant arguments.
* Arguments are passed without modifications, useDefaultImplementationForNulls, useDefaultImplementationForConstants,
* useDefaultImplementationForLowCardinality are not applied.
* Arguments are passed without modifications, useDefaultImplementationForNulls, useDefaultImplementationForNothing,
* useDefaultImplementationForConstants, useDefaultImplementationForLowCardinality are not applied.
*/
virtual ColumnPtr getConstantResultForNonConstArguments(
const ColumnsWithTypeAndName & /* arguments */, const DataTypePtr & /* result_type */) const { return nullptr; }
@ -354,7 +362,13 @@ protected:
*/
virtual bool useDefaultImplementationForNulls() const { return true; }
/** If useDefaultImplementationForNulls() is true, then change arguments for getReturnType() and build().
/** If useDefaultImplementationForNothing() is true, then change arguments for getReturnType() and build():
* if some of arguments are Nothing then don't call getReturnType(), call build() with return_type = Nothing,
* Otherwise build returns build(arguments, getReturnType(arguments));
*/
virtual bool useDefaultImplementationForNothing() const { return true; }
/** If useDefaultImplementationForLowCardinalityColumns() is true, then change arguments for getReturnType() and build().
* If function arguments has low cardinality types, convert them to ordinary types.
* getReturnType returns ColumnLowCardinality if at least one argument type is ColumnLowCardinality.
*/
@ -403,6 +417,11 @@ public:
*/
virtual bool useDefaultImplementationForNulls() const { return true; }
/** Default implementation in presence of arguments with type Nothing is the following:
* If some of arguments have type Nothing then default implementation is to return constant column with type Nothing
*/
virtual bool useDefaultImplementationForNothing() const { return true; }
/** If the function have non-zero number of arguments,
* and if all arguments are constant, that we could automatically provide default implementation:
* arguments are converted to ordinary columns with single value, then function is executed as usual,

View File

@ -27,6 +27,7 @@ protected:
}
bool useDefaultImplementationForNulls() const final { return function->useDefaultImplementationForNulls(); }
bool useDefaultImplementationForNothing() const final { return function->useDefaultImplementationForNothing(); }
bool useDefaultImplementationForConstants() const final { return function->useDefaultImplementationForConstants(); }
bool useDefaultImplementationForLowCardinalityColumns() const final { return function->useDefaultImplementationForLowCardinalityColumns(); }
bool useDefaultImplementationForSparseColumns() const final { return function->useDefaultImplementationForSparseColumns(); }
@ -124,6 +125,7 @@ public:
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { return function->getReturnTypeImpl(arguments); }
bool useDefaultImplementationForNulls() const override { return function->useDefaultImplementationForNulls(); }
bool useDefaultImplementationForNothing() const override { return function->useDefaultImplementationForNothing(); }
bool useDefaultImplementationForLowCardinalityColumns() const override { return function->useDefaultImplementationForLowCardinalityColumns(); }
bool useDefaultImplementationForSparseColumns() const override { return function->useDefaultImplementationForSparseColumns(); }
bool canBeExecutedOnLowCardinalityDictionary() const override { return function->canBeExecutedOnLowCardinalityDictionary(); }

View File

@ -63,14 +63,33 @@ inline bool likePatternIsSubstring(std::string_view pattern, String & res)
}
/** 'like' - if true, treat pattern as SQL LIKE, otherwise as re2 regexp.
* 'negate' - if true, negate result
* 'case_insensitive' - if true, match case insensitively
*
* NOTE: We want to run regexp search for whole columns by one call (as implemented in function 'position')
* but for that, regexp engine must support \0 bytes and their interpretation as string boundaries.
*/
template <typename Name, bool like, bool negate, bool case_insensitive>
// For more readable instantiations of MatchImpl<>
struct MatchTraits
{
enum class Syntax
{
Like,
Re2
};
enum class Case
{
Sensitive,
Insensitive
};
enum class Result
{
DontNegate,
Negate
};
};
/**
* NOTE: We want to run regexp search for whole columns by one call (as implemented in function 'position')
* but for that, regexp engine must support \0 bytes and their interpretation as string boundaries.
*/
template <typename Name, MatchTraits::Syntax syntax_, MatchTraits::Case case_, MatchTraits::Result result_>
struct MatchImpl
{
static constexpr bool use_default_implementation_for_constants = true;
@ -81,6 +100,10 @@ struct MatchImpl
using ResultType = UInt8;
static constexpr bool is_like = (syntax_ == MatchTraits::Syntax::Like);
static constexpr bool case_insensitive = (case_ == MatchTraits::Case::Insensitive);
static constexpr bool negate = (result_ == MatchTraits::Result::Negate);
using Searcher = std::conditional_t<case_insensitive,
VolnitskyCaseInsensitiveUTF8,
VolnitskyUTF8>;
@ -92,16 +115,20 @@ struct MatchImpl
const ColumnPtr & start_pos_,
PaddedPODArray<UInt8> & res)
{
const size_t haystack_size = haystack_offsets.size();
if (haystack_size != res.size())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Function '{}' unexpectedly received a different number of haystacks and results", name);
if (start_pos_ != nullptr)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Function '{}' doesn't support start_pos argument", name);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function '{}' doesn't support start_pos argument", name);
if (haystack_offsets.empty())
return;
/// A simple case where the [I]LIKE expression reduces to finding a substring in a string
String strstr_pattern;
if (like && impl::likePatternIsSubstring(needle, strstr_pattern))
if (is_like && impl::likePatternIsSubstring(needle, strstr_pattern))
{
const UInt8 * const begin = haystack_data.data();
const UInt8 * const end = haystack_data.data() + haystack_data.size();
@ -139,7 +166,7 @@ struct MatchImpl
}
else
{
auto regexp = Regexps::get<like, true, case_insensitive>(needle);
auto regexp = Regexps::get<is_like, true, case_insensitive>(needle);
String required_substring;
bool is_trivial;
@ -147,28 +174,26 @@ struct MatchImpl
regexp->getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix);
size_t haystack_size = haystack_offsets.size();
if (required_substring.empty())
{
if (!regexp->getRE2()) /// An empty regexp. Always matches.
{
if (haystack_size)
memset(res.data(), 1, haystack_size * sizeof(res[0]));
memset(res.data(), !negate, haystack_size * sizeof(res[0]));
}
else
{
size_t prev_offset = 0;
for (size_t i = 0; i < haystack_size; ++i)
{
res[i] = negate
^ regexp->getRE2()->Match(
{reinterpret_cast<const char *>(&haystack_data[prev_offset]), haystack_offsets[i] - prev_offset - 1},
0,
haystack_offsets[i] - prev_offset - 1,
re2_st::RE2::UNANCHORED,
nullptr,
0);
const bool match = regexp->getRE2()->Match(
{reinterpret_cast<const char *>(&haystack_data[prev_offset]), haystack_offsets[i] - prev_offset - 1},
0,
haystack_offsets[i] - prev_offset - 1,
re2_st::RE2::UNANCHORED,
nullptr,
0);
res[i] = negate ^ match;
prev_offset = haystack_offsets[i];
}
@ -216,14 +241,14 @@ struct MatchImpl
const size_t start_pos = (required_substring_is_prefix) ? (reinterpret_cast<const char *>(pos) - str_data) : 0;
const size_t end_pos = str_size;
res[i] = negate
^ regexp->getRE2()->Match(
{str_data, str_size},
start_pos,
end_pos,
re2_st::RE2::UNANCHORED,
nullptr,
0);
const bool match = regexp->getRE2()->Match(
{str_data, str_size},
start_pos,
end_pos,
re2_st::RE2::UNANCHORED,
nullptr,
0);
res[i] = negate ^ match;
}
}
else
@ -247,12 +272,17 @@ struct MatchImpl
const String & needle,
PaddedPODArray<UInt8> & res)
{
const size_t haystack_size = haystack.size() / N;
if (haystack_size != res.size())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Function '{}' unexpectedly received a different number of haystacks and results", name);
if (haystack.empty())
return;
/// A simple case where the LIKE expression reduces to finding a substring in a string
String strstr_pattern;
if (like && impl::likePatternIsSubstring(needle, strstr_pattern))
if (is_like && impl::likePatternIsSubstring(needle, strstr_pattern))
{
const UInt8 * const begin = haystack.data();
const UInt8 * const end = haystack.data() + haystack.size();
@ -295,7 +325,7 @@ struct MatchImpl
}
else
{
auto regexp = Regexps::get<like, true, case_insensitive>(needle);
auto regexp = Regexps::get<is_like, true, case_insensitive>(needle);
String required_substring;
bool is_trivial;
@ -303,28 +333,26 @@ struct MatchImpl
regexp->getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix);
const size_t haystack_size = haystack.size() / N;
if (required_substring.empty())
{
if (!regexp->getRE2()) /// An empty regexp. Always matches.
{
if (haystack_size)
memset(res.data(), 1, haystack_size * sizeof(res[0]));
memset(res.data(), !negate, haystack_size * sizeof(res[0]));
}
else
{
size_t offset = 0;
for (size_t i = 0; i < haystack_size; ++i)
{
res[i] = negate
^ regexp->getRE2()->Match(
{reinterpret_cast<const char *>(&haystack[offset]), N},
0,
N,
re2_st::RE2::UNANCHORED,
nullptr,
0);
const bool match = regexp->getRE2()->Match(
{reinterpret_cast<const char *>(&haystack[offset]), N},
0,
N,
re2_st::RE2::UNANCHORED,
nullptr,
0);
res[i] = negate ^ match;
offset += N;
}
@ -375,14 +403,14 @@ struct MatchImpl
const size_t start_pos = (required_substring_is_prefix) ? (reinterpret_cast<const char *>(pos) - str_data) : 0;
const size_t end_pos = N;
res[i] = negate
^ regexp->getRE2()->Match(
const bool match = regexp->getRE2()->Match(
{str_data, N},
start_pos,
end_pos,
re2_st::RE2::UNANCHORED,
nullptr,
0);
res[i] = negate ^ match;
}
}
else
@ -410,13 +438,11 @@ struct MatchImpl
{
const size_t haystack_size = haystack_offsets.size();
if (haystack_size != needle_offset.size())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Function '{}' unexpectedly received a different number of haystacks and needles", name);
if (haystack_size != needle_offset.size() || haystack_size != res.size())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Function '{}' unexpectedly received a different number of haystacks, needles and results", name);
if (start_pos_ != nullptr)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Function '{}' doesn't support start_pos argument", name);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function '{}' doesn't support start_pos argument", name);
if (haystack_offsets.empty())
return;
@ -440,7 +466,7 @@ struct MatchImpl
reinterpret_cast<const char *>(cur_needle_data),
cur_needle_length);
if (like && impl::likePatternIsSubstring(needle, required_substr))
if (is_like && impl::likePatternIsSubstring(needle, required_substr))
{
if (required_substr.size() > cur_haystack_length)
res[i] = negate;
@ -448,16 +474,15 @@ struct MatchImpl
{
Searcher searcher(required_substr.data(), required_substr.size(), cur_haystack_length);
const auto * match = searcher.search(cur_haystack_data, cur_haystack_length);
res[i] = negate
^ (match != cur_haystack_data + cur_haystack_length);
res[i] = negate ^ (match != cur_haystack_data + cur_haystack_length);
}
}
else
{
// each row is expected to contain a different like/re2 pattern
// --> bypass the regexp cache, instead construct the pattern on-the-fly
const int flags = Regexps::buildRe2Flags<true, case_insensitive>();
const auto & regexp = Regexps::Regexp(Regexps::createRegexp<like>(needle, flags));
const int flags = Regexps::buildRe2Flags</*no_capture*/ true, case_insensitive>();
const auto & regexp = Regexps::Regexp(Regexps::createRegexp<is_like>(needle, flags));
regexp.getAnalyzeResult(required_substr, is_trivial, required_substring_is_prefix);
@ -465,18 +490,18 @@ struct MatchImpl
{
if (!regexp.getRE2()) /// An empty regexp. Always matches.
{
res[i] = 1;
res[i] = !negate;
}
else
{
res[i] = negate
^ regexp.getRE2()->Match(
{reinterpret_cast<const char *>(cur_haystack_data), cur_haystack_length},
0,
cur_haystack_length,
re2_st::RE2::UNANCHORED,
nullptr,
0);
const bool match = regexp.getRE2()->Match(
{reinterpret_cast<const char *>(cur_haystack_data), cur_haystack_length},
0,
cur_haystack_length,
re2_st::RE2::UNANCHORED,
nullptr,
0);
res[i] = negate ^ match;
}
}
else
@ -499,14 +524,14 @@ struct MatchImpl
const size_t start_pos = (required_substring_is_prefix) ? (match - cur_haystack_data) : 0;
const size_t end_pos = cur_haystack_length;
res[i] = negate
^ regexp.getRE2()->Match(
{reinterpret_cast<const char *>(cur_haystack_data), cur_haystack_length},
start_pos,
end_pos,
re2_st::RE2::UNANCHORED,
nullptr,
0);
const bool match2 = regexp.getRE2()->Match(
{reinterpret_cast<const char *>(cur_haystack_data), cur_haystack_length},
start_pos,
end_pos,
re2_st::RE2::UNANCHORED,
nullptr,
0);
res[i] = negate ^ match2;
}
}
}
@ -527,13 +552,11 @@ struct MatchImpl
{
const size_t haystack_size = haystack.size()/N;
if (haystack_size != needle_offset.size())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Function '{}' unexpectedly received a different number of haystacks and needles", name);
if (haystack_size != needle_offset.size() || haystack_size != res.size())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Function '{}' unexpectedly received a different number of haystacks, needles and results", name);
if (start_pos_ != nullptr)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Function '{}' doesn't support start_pos argument", name);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function '{}' doesn't support start_pos argument", name);
if (haystack.empty())
return;
@ -557,7 +580,7 @@ struct MatchImpl
reinterpret_cast<const char *>(cur_needle_data),
cur_needle_length);
if (like && impl::likePatternIsSubstring(needle, required_substr))
if (is_like && impl::likePatternIsSubstring(needle, required_substr))
{
if (required_substr.size() > cur_haystack_length)
res[i] = negate;
@ -565,16 +588,15 @@ struct MatchImpl
{
Searcher searcher(required_substr.data(), required_substr.size(), cur_haystack_length);
const auto * match = searcher.search(cur_haystack_data, cur_haystack_length);
res[i] = negate
^ (match != cur_haystack_data + cur_haystack_length);
res[i] = negate ^ (match != cur_haystack_data + cur_haystack_length);
}
}
else
{
// each row is expected to contain a different like/re2 pattern
// --> bypass the regexp cache, instead construct the pattern on-the-fly
const int flags = Regexps::buildRe2Flags<true, case_insensitive>();
const auto & regexp = Regexps::Regexp(Regexps::createRegexp<like>(needle, flags));
const int flags = Regexps::buildRe2Flags</*no_capture*/ true, case_insensitive>();
const auto & regexp = Regexps::Regexp(Regexps::createRegexp<is_like>(needle, flags));
regexp.getAnalyzeResult(required_substr, is_trivial, required_substring_is_prefix);
@ -582,18 +604,18 @@ struct MatchImpl
{
if (!regexp.getRE2()) /// An empty regexp. Always matches.
{
res[i] = 1;
res[i] = !negate;
}
else
{
res[i] = negate
^ regexp.getRE2()->Match(
{reinterpret_cast<const char *>(cur_haystack_data), cur_haystack_length},
0,
cur_haystack_length,
re2_st::RE2::UNANCHORED,
nullptr,
0);
const bool match = regexp.getRE2()->Match(
{reinterpret_cast<const char *>(cur_haystack_data), cur_haystack_length},
0,
cur_haystack_length,
re2_st::RE2::UNANCHORED,
nullptr,
0);
res[i] = negate ^ match;
}
}
else
@ -616,14 +638,14 @@ struct MatchImpl
const size_t start_pos = (required_substring_is_prefix) ? (match - cur_haystack_data) : 0;
const size_t end_pos = cur_haystack_length;
res[i] = negate
^ regexp.getRE2()->Match(
{reinterpret_cast<const char *>(cur_haystack_data), cur_haystack_length},
start_pos,
end_pos,
re2_st::RE2::UNANCHORED,
nullptr,
0);
const bool match2 = regexp.getRE2()->Match(
{reinterpret_cast<const char *>(cur_haystack_data), cur_haystack_length},
start_pos,
end_pos,
re2_st::RE2::UNANCHORED,
nullptr,
0);
res[i] = negate ^ match2;
}
}
}

View File

@ -120,7 +120,7 @@ struct MultiMatchAnyImpl
memset(accum.data(), 0, accum.size());
for (size_t j = 0; j < needles.size(); ++j)
{
MatchImpl<Name, false, false, false>::vectorConstant(haystack_data, haystack_offsets, needles[j].toString(), nullptr, accum);
MatchImpl<Name, MatchTraits::Syntax::Re2, MatchTraits::Case::Sensitive, MatchTraits::Result::DontNegate>::vectorConstant(haystack_data, haystack_offsets, needles[j].toString(), nullptr, accum);
for (size_t i = 0; i < res.size(); ++i)
{
if constexpr (FindAny)

View File

@ -6,6 +6,7 @@
#include <Columns/ColumnConst.h>
#include <Columns/ColumnFunction.h>
#include <Columns/ColumnMap.h>
#include <Columns/ColumnNullable.h>
#include <Columns/IColumn.h>
#include <Common/Exception.h>
@ -16,11 +17,13 @@
#include <DataTypes/DataTypeFunction.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <Interpreters/Context_fwd.h>
#include <Interpreters/castColumn.h>
#include <IO/WriteHelpers.h>
@ -156,7 +159,7 @@ public:
DataTypePtr nested_type = data_type->getNestedType();
if (Impl::needBoolean() && !WhichDataType(nested_type).isUInt8())
if (Impl::needBoolean() && !isUInt8(nested_type))
throw Exception("The only argument for function " + getName() + " must be array of UInt8. Found "
+ arguments[0].type->getName() + " instead", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -180,8 +183,14 @@ public:
/// The types of the remaining arguments are already checked in getLambdaArgumentTypes.
DataTypePtr return_type = removeLowCardinality(data_type_function->getReturnType());
if (Impl::needBoolean() && !WhichDataType(return_type).isUInt8())
throw Exception("Expression for function " + getName() + " must return UInt8, found "
/// Special cases when we need boolean lambda result:
/// - lambda may return Nullable(UInt8) column, in this case after lambda execution we will
/// replace all NULLs with 0 and return nested UInt8 column.
/// - lambda may return Nothing or Nullable(Nothing) because of default implementation of functions
/// for these types. In this case we will just create UInt8 const column full of 0.
if (Impl::needBoolean() && !isUInt8(removeNullable(return_type)) && !isNothing(removeNullable(return_type)))
throw Exception("Expression for function " + getName() + " must return UInt8 or Nullable(UInt8), found "
+ return_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
static_assert(
@ -316,11 +325,37 @@ public:
auto * replicated_column_function = typeid_cast<ColumnFunction *>(replicated_column_function_ptr.get());
replicated_column_function->appendArguments(arrays);
auto lambda_result = replicated_column_function->reduce().column;
if (lambda_result->lowCardinality())
lambda_result = lambda_result->convertToFullColumnIfLowCardinality();
auto lambda_result = replicated_column_function->reduce();
if (lambda_result.column->lowCardinality())
lambda_result.column = lambda_result.column->convertToFullColumnIfLowCardinality();
return Impl::execute(*column_first_array, lambda_result);
if (Impl::needBoolean())
{
/// If result column is Nothing or Nullable(Nothing), just create const UInt8 column with 0 value.
if (isNothing(removeNullable(lambda_result.type)))
{
auto result_type = std::make_shared<DataTypeUInt8>();
lambda_result.column = result_type->createColumnConst(lambda_result.column->size(), 0);
}
/// If result column is Nullable(UInt8), then extract nested column and write 0 in all rows
/// when we have NULL.
else if (lambda_result.column->isNullable())
{
auto result_column = IColumn::mutate(std::move(lambda_result.column));
auto * column_nullable = assert_cast<ColumnNullable *>(result_column.get());
auto & null_map = column_nullable->getNullMapData();
auto nested_column = IColumn::mutate(std::move(column_nullable->getNestedColumnPtr()));
auto & nested_data = assert_cast<ColumnUInt8 *>(nested_column.get())->getData();
for (size_t i = 0; i != nested_data.size(); ++i)
{
if (null_map[i])
nested_data[i] = 0;
}
lambda_result.column = std::move(nested_column);
}
}
return Impl::execute(*column_first_array, lambda_result.column);
}
}
};

View File

@ -20,6 +20,8 @@ public:
}
bool useDefaultImplementationForNulls() const override { return false; }
/// array(..., Nothing, ...) -> Array(..., Nothing, ...)
bool useDefaultImplementationForNothing() const override { return false; }
bool useDefaultImplementationForConstants() const override { return true; }
bool isVariadic() const override { return true; }

View File

@ -7,6 +7,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
namespace
{
@ -45,6 +51,9 @@ public:
{
const ColumnPtr & col = arguments[0].column;
if (arguments[0].type->onlyNull() && !col->empty())
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot create non-empty column with type Nothing");
if (const auto * nullable_col = checkAndGetColumn<ColumnNullable>(*col))
return nullable_col->getNestedColumnPtr();
else

View File

@ -52,6 +52,7 @@ public:
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForNothing() const override { return false; }
bool useDefaultImplementationForConstants() const override { return false; }
bool useDefaultImplementationForLowCardinalityColumns() const override { return true; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
@ -194,6 +195,7 @@ private:
bool isVariadic() const override { return true; }
bool useDefaultImplementationForNulls() const override { return impl.useDefaultImplementationForNulls(); }
bool useDefaultImplementationForNothing() const override { return impl.useDefaultImplementationForNothing(); }
bool useDefaultImplementationForLowCardinalityColumns() const override { return impl.useDefaultImplementationForLowCardinalityColumns();}
bool useDefaultImplementationForConstants() const override { return impl.useDefaultImplementationForConstants();}
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & arguments) const override

View File

@ -7,6 +7,7 @@
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionFactory.h>
#include <Functions/PerformanceAdaptors.h>
#include <Interpreters/castColumn.h>
#include <Common/TargetSpecific.h>
#include <base/range.h>
#include <cmath>
@ -14,9 +15,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
/** Calculates the distance between two geographical locations.
@ -262,24 +265,48 @@ private:
return std::make_shared<DataTypeFloat32>();
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
auto dst = ColumnVector<Float32>::create();
auto & dst_data = dst->getData();
dst_data.resize(input_rows_count);
const IColumn & col_lon1 = *arguments[0].column;
const IColumn & col_lat1 = *arguments[1].column;
const IColumn & col_lon2 = *arguments[2].column;
const IColumn & col_lat2 = *arguments[3].column;
auto arguments_copy = arguments;
for (auto & argument : arguments_copy)
{
argument.column = argument.column->convertToFullColumnIfConst();
argument.column = castColumn(argument, result_type);
argument.type = result_type;
}
const auto * col_lon1 = convertArgumentColumnToFloat32(arguments_copy, 0);
const auto * col_lat1 = convertArgumentColumnToFloat32(arguments_copy, 1);
const auto * col_lon2 = convertArgumentColumnToFloat32(arguments_copy, 2);
const auto * col_lat2 = convertArgumentColumnToFloat32(arguments_copy, 3);
for (size_t row_num = 0; row_num < input_rows_count; ++row_num)
{
dst_data[row_num] = distance<method>(
col_lon1.getFloat32(row_num), col_lat1.getFloat32(row_num),
col_lon2.getFloat32(row_num), col_lat2.getFloat32(row_num));
col_lon1->getData()[row_num], col_lat1->getData()[row_num],
col_lon2->getData()[row_num], col_lat2->getData()[row_num]);
}
return dst;
}
const ColumnFloat32 * convertArgumentColumnToFloat32(const ColumnsWithTypeAndName & arguments, size_t argument_index) const
{
const auto * column_typed = checkAndGetColumn<ColumnFloat32>(arguments[argument_index].column.get());
if (!column_typed)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be Float32.",
arguments[argument_index].type->getName(),
argument_index + 1,
getName());
return column_typed;
}
};
) // DECLARE_MULTITARGET_CODE

View File

@ -0,0 +1,95 @@
#include "config_functions.h"
#if USE_H3
#include <Columns/ColumnArray.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <Common/typeid_cast.h>
#include <IO/WriteHelpers.h>
#include <base/range.h>
#include <constants.h>
#include <h3api.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
namespace
{
class FunctionH3GetDestinationIndexFromUnidirectionalEdge : public IFunction
{
public:
static constexpr auto name = "h3GetDestinationIndexFromUnidirectionalEdge";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionH3GetDestinationIndexFromUnidirectionalEdge>(); }
std::string getName() const override { return name; }
size_t getNumberOfArguments() const override { return 1; }
bool useDefaultImplementationForConstants() const override { return true; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const auto * arg = arguments[0].get();
if (!WhichDataType(arg).isUInt64())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt64",
arg->getName(), 1, getName());
return std::make_shared<DataTypeUInt64>();
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
auto non_const_arguments = arguments;
for (auto & argument : non_const_arguments)
argument.column = argument.column->convertToFullColumnIfConst();
const auto * col_hindex_edge = checkAndGetColumn<ColumnUInt64>(non_const_arguments[0].column.get());
if (!col_hindex_edge)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt64.",
arguments[0].type->getName(),
1,
getName());
const auto & data_hindex_edge = col_hindex_edge->getData();
auto dst = ColumnVector<UInt64>::create();
auto & dst_data = dst->getData();
dst_data.resize(input_rows_count);
for (size_t row = 0; row < input_rows_count; ++row)
{
const UInt64 edge = data_hindex_edge[row];
const UInt64 res = getDirectedEdgeDestination(edge);
dst_data[row] = res;
}
return dst;
}
};
}
void registerFunctionH3GetDestinationIndexFromUnidirectionalEdge(FunctionFactory & factory)
{
factory.registerFunction<FunctionH3GetDestinationIndexFromUnidirectionalEdge>();
}
}
#endif

View File

@ -0,0 +1,110 @@
#include "config_functions.h"
#if USE_H3
#include <Columns/ColumnArray.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnTuple.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <Common/typeid_cast.h>
#include <IO/WriteHelpers.h>
#include <h3api.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
namespace
{
class FunctionH3GetIndexesFromUnidirectionalEdge : public IFunction
{
public:
static constexpr auto name = "h3GetIndexesFromUnidirectionalEdge";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionH3GetIndexesFromUnidirectionalEdge>(); }
std::string getName() const override { return name; }
size_t getNumberOfArguments() const override { return 1; }
bool useDefaultImplementationForConstants() const override { return true; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const auto * arg = arguments[0].get();
if (!WhichDataType(arg).isUInt64())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt64",
arg->getName(), 1, getName());
return std::make_shared<DataTypeTuple>(
DataTypes{std::make_shared<DataTypeUInt64>(), std::make_shared<DataTypeUInt64>()},
Strings{"origin", "destination"});
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
auto non_const_arguments = arguments;
for (auto & argument : non_const_arguments)
argument.column = argument.column->convertToFullColumnIfConst();
const auto * col_hindex_edge = checkAndGetColumn<ColumnUInt64>(non_const_arguments[0].column.get());
if (!col_hindex_edge)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt64.",
arguments[0].type->getName(),
1,
getName());
const auto & data_hindex_edge = col_hindex_edge->getData();
auto origin = ColumnUInt64::create(input_rows_count);
auto destination = ColumnUInt64::create(input_rows_count);
ColumnUInt64::Container & origin_data = origin->getData();
ColumnUInt64::Container & destination_data = destination->getData();
for (size_t row = 0; row < input_rows_count; ++row)
{
const UInt64 edge = data_hindex_edge[row];
// allocate array of size 2
// directedEdgeToCells func sets the origin and
// destination at [0] and [1] of the input vector
std::array<H3Index, 2> res;
directedEdgeToCells(edge, res.data());
origin_data[row] = res[0];
destination_data[row] = res[1];
}
MutableColumns columns;
columns.emplace_back(std::move(origin));
columns.emplace_back(std::move(destination));
return ColumnTuple::create(std::move(columns));
}
};
}
void registerFunctionH3GetIndexesFromUnidirectionalEdge(FunctionFactory & factory)
{
factory.registerFunction<FunctionH3GetIndexesFromUnidirectionalEdge>();
}
}
#endif

View File

@ -0,0 +1,95 @@
#include "config_functions.h"
#if USE_H3
#include <Columns/ColumnArray.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <Common/typeid_cast.h>
#include <IO/WriteHelpers.h>
#include <base/range.h>
#include <constants.h>
#include <h3api.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
namespace
{
class FunctionH3GetOriginIndexFromUnidirectionalEdge : public IFunction
{
public:
static constexpr auto name = "h3GetOriginIndexFromUnidirectionalEdge";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionH3GetOriginIndexFromUnidirectionalEdge>(); }
std::string getName() const override { return name; }
size_t getNumberOfArguments() const override { return 1; }
bool useDefaultImplementationForConstants() const override { return true; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const auto * arg = arguments[0].get();
if (!WhichDataType(arg).isUInt64())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt64",
arg->getName(), 1, getName());
return std::make_shared<DataTypeUInt64>();
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
auto non_const_arguments = arguments;
for (auto & argument : non_const_arguments)
argument.column = argument.column->convertToFullColumnIfConst();
const auto * col_hindex_edge = checkAndGetColumn<ColumnUInt64>(non_const_arguments[0].column.get());
if (!col_hindex_edge)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt64.",
arguments[0].type->getName(),
1,
getName());
const auto & data_hindex_edge = col_hindex_edge->getData();
auto dst = ColumnVector<UInt64>::create();
auto & dst_data = dst->getData();
dst_data.resize(input_rows_count);
for (size_t row = 0; row < input_rows_count; ++row)
{
const UInt64 edge = data_hindex_edge[row];
const UInt64 res = getDirectedEdgeOrigin(edge);
dst_data[row] = res;
}
return dst;
}
};
}
void registerFunctionH3GetOriginIndexFromUnidirectionalEdge(FunctionFactory & factory)
{
factory.registerFunction<FunctionH3GetOriginIndexFromUnidirectionalEdge>();
}
}
#endif

View File

@ -0,0 +1,127 @@
#include "config_functions.h"
#if USE_H3
#include <Columns/ColumnArray.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <Common/typeid_cast.h>
#include <IO/WriteHelpers.h>
#include <h3api.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
namespace
{
class FunctionH3GetUnidirectionalEdge : public IFunction
{
public:
static constexpr auto name = "h3GetUnidirectionalEdge";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionH3GetUnidirectionalEdge>(); }
std::string getName() const override { return name; }
size_t getNumberOfArguments() const override { return 2; }
bool useDefaultImplementationForConstants() const override { return true; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const auto * arg = arguments[0].get();
if (!WhichDataType(arg).isUInt64())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt64",
arg->getName(), 1, getName());
arg = arguments[1].get();
if (!WhichDataType(arg).isUInt64())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt64",
arg->getName(), 2, getName());
return std::make_shared<DataTypeUInt64>();
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
auto non_const_arguments = arguments;
for (auto & argument : non_const_arguments)
argument.column = argument.column->convertToFullColumnIfConst();
const auto * col_hindex_origin = checkAndGetColumn<ColumnUInt64>(non_const_arguments[0].column.get());
if (!col_hindex_origin)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt64.",
arguments[0].type->getName(),
1,
getName());
const auto & data_hindex_origin = col_hindex_origin->getData();
const auto * col_hindex_dest = checkAndGetColumn<ColumnUInt64>(non_const_arguments[1].column.get());
if (!col_hindex_dest)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt64.",
arguments[1].type->getName(),
2,
getName());
const auto & data_hindex_dest = col_hindex_dest->getData();
auto dst = ColumnVector<UInt64>::create();
auto & dst_data = dst->getData();
dst_data.resize(input_rows_count);
for (size_t row = 0; row < input_rows_count; ++row)
{
const UInt64 origin = data_hindex_origin[row];
const UInt64 dest = data_hindex_dest[row];
if (!isValidCell(origin))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Invalid origin H3 index: {}", origin);
if (!isValidCell(dest))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Invalid dest H3 index: {}", dest);
UInt64 res = getUnidirectionalEdge(origin, dest);
dst_data[row] = res;
}
return dst;
}
/// suppress asan errors generated by the following:
/// 'NEW_ADJUSTMENT_III' defined in '../contrib/h3/src/h3lib/lib/algos.c:142:24
/// 'NEW_DIGIT_III' defined in '../contrib/h3/src/h3lib/lib/algos.c:121:24
__attribute__((no_sanitize_address)) static inline UInt64 getUnidirectionalEdge(const UInt64 origin, const UInt64 dest)
{
const UInt64 res = cellsToDirectedEdge(origin, dest);
return res;
}
};
}
void registerFunctionH3GetUnidirectionalEdge(FunctionFactory & factory)
{
factory.registerFunction<FunctionH3GetUnidirectionalEdge>();
}
}
#endif

View File

@ -0,0 +1,111 @@
#include "config_functions.h"
#if USE_H3
#include <Columns/ColumnArray.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnTuple.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <Common/typeid_cast.h>
#include <IO/WriteHelpers.h>
#include <h3api.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
namespace
{
class FunctionH3GetUnidirectionalEdgeBoundary : public IFunction
{
public:
static constexpr auto name = "h3GetUnidirectionalEdgeBoundary";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionH3GetUnidirectionalEdgeBoundary>(); }
std::string getName() const override { return name; }
size_t getNumberOfArguments() const override { return 1; }
bool useDefaultImplementationForConstants() const override { return true; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const auto * arg = arguments[0].get();
if (!WhichDataType(arg).isUInt64())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt64",
arg->getName(), 1, getName());
return std::make_shared<DataTypeArray>(
std::make_shared<DataTypeTuple>(
DataTypes{std::make_shared<DataTypeFloat64>(), std::make_shared<DataTypeFloat64>()}));
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
auto non_const_arguments = arguments;
for (auto & argument : non_const_arguments)
argument.column = argument.column->convertToFullColumnIfConst();
const auto * col_hindex_edge = checkAndGetColumn<ColumnUInt64>(non_const_arguments[0].column.get());
if (!col_hindex_edge)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt64.",
arguments[0].type->getName(),
1,
getName());
const auto & data_hindex_edge = col_hindex_edge->getData();
auto latitude = ColumnFloat64::create();
auto longitude = ColumnFloat64::create();
auto offsets = DataTypeNumber<IColumn::Offset>().createColumn();
offsets->reserve(input_rows_count);
IColumn::Offset current_offset = 0;
for (size_t row = 0; row < input_rows_count; ++row)
{
H3Index edge = data_hindex_edge[row];
CellBoundary boundary{};
directedEdgeToBoundary(edge, &boundary);
for (int vert = 0; vert < boundary.numVerts; ++vert)
{
latitude->getData().push_back(radsToDegs(boundary.verts[vert].lat));
longitude->getData().push_back(radsToDegs(boundary.verts[vert].lng));
}
current_offset += boundary.numVerts;
offsets->insert(current_offset);
}
return ColumnArray::create(
ColumnTuple::create(Columns{std::move(latitude), std::move(longitude)}),
std::move(offsets));
}
};
}
void registerFunctionH3GetUnidirectionalEdgeBoundary(FunctionFactory & factory)
{
factory.registerFunction<FunctionH3GetUnidirectionalEdgeBoundary>();
}
}
#endif

View File

@ -0,0 +1,112 @@
#include "config_functions.h"
#if USE_H3
#include <Columns/ColumnArray.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnTuple.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <Common/typeid_cast.h>
#include <IO/WriteHelpers.h>
#include <h3api.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
namespace
{
class FunctionH3GetUnidirectionalEdgesFromHexagon : public IFunction
{
public:
static constexpr auto name = "h3GetUnidirectionalEdgesFromHexagon";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionH3GetUnidirectionalEdgesFromHexagon>(); }
std::string getName() const override { return name; }
size_t getNumberOfArguments() const override { return 1; }
bool useDefaultImplementationForConstants() const override { return true; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const auto * arg = arguments[0].get();
if (!WhichDataType(arg).isUInt64())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt64",
arg->getName(), 1, getName());
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
auto non_const_arguments = arguments;
for (auto & argument : non_const_arguments)
argument.column = argument.column->convertToFullColumnIfConst();
const auto * col_hindex_edge = checkAndGetColumn<ColumnUInt64>(non_const_arguments[0].column.get());
if (!col_hindex_edge)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt64.",
arguments[0].type->getName(),
1,
getName());
const auto & data_hindex_edge = col_hindex_edge->getData();
auto result_column_data = ColumnUInt64::create();
auto & result_data = result_column_data->getData();
auto result_column_offsets = ColumnArray::ColumnOffsets::create();
auto & result_offsets = result_column_offsets->getData();
result_offsets.resize(input_rows_count);
auto current_offset = 0;
result_data.reserve(input_rows_count);
for (size_t row = 0; row < input_rows_count; ++row)
{
// allocate array of size 6
// originToDirectedEdges places 6 edges into
// array that's passed to it
std::array<H3Index, 6> res;
const UInt64 edge = data_hindex_edge[row];
originToDirectedEdges(edge, res.data());
for (auto & i : res)
{
++current_offset;
result_data.emplace_back(i);
}
result_offsets[row] = current_offset;
}
return ColumnArray::create(std::move(result_column_data), std::move(result_column_offsets));
}
};
}
void registerFunctionH3GetUnidirectionalEdgesFromHexagon(FunctionFactory & factory)
{
factory.registerFunction<FunctionH3GetUnidirectionalEdgesFromHexagon>();
}
}
#endif

View File

@ -0,0 +1,97 @@
#include "config_functions.h"
#if USE_H3
#include <Columns/ColumnArray.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <Common/typeid_cast.h>
#include <IO/WriteHelpers.h>
#include <base/range.h>
#include <constants.h>
#include <h3api.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
namespace
{
class FunctionH3UnidirectionalEdgeIsValid : public IFunction
{
public:
static constexpr auto name = "h3UnidirectionalEdgeIsValid";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionH3UnidirectionalEdgeIsValid>(); }
std::string getName() const override { return name; }
size_t getNumberOfArguments() const override { return 1; }
bool useDefaultImplementationForConstants() const override { return true; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const auto * arg = arguments[0].get();
if (!WhichDataType(arg).isUInt64())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of function {}. Must be UInt64",
arg->getName(),
1,
getName());
return std::make_shared<DataTypeUInt8>();
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
auto non_const_arguments = arguments;
for (auto & argument : non_const_arguments)
argument.column = argument.column->convertToFullColumnIfConst();
const auto * col_hindex_edge = checkAndGetColumn<ColumnUInt64>(non_const_arguments[0].column.get());
if (!col_hindex_edge)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of argument {} of function {}. Must be UInt64.",
arguments[0].type->getName(),
1,
getName());
const auto & data_hindex_edge = col_hindex_edge->getData();
auto dst = ColumnVector<UInt8>::create();
auto & dst_data = dst->getData();
dst_data.resize(input_rows_count);
for (size_t row = 0; row < input_rows_count; ++row)
{
const UInt64 edge = data_hindex_edge[row];
const UInt8 res = isValidDirectedEdge(edge);
dst_data[row] = res;
}
return dst;
}
};
}
void registerFunctionH3UnidirectionalEdgeIsValid(FunctionFactory & factory)
{
factory.registerFunction<FunctionH3UnidirectionalEdgeIsValid>();
}
}
#endif

View File

@ -12,7 +12,7 @@ struct NameILike
static constexpr auto name = "ilike";
};
using ILikeImpl = MatchImpl<NameILike, true, false, true>;
using ILikeImpl = MatchImpl<NameILike, MatchTraits::Syntax::Like, MatchTraits::Case::Insensitive, MatchTraits::Result::DontNegate>;
using FunctionILike = FunctionsStringSearch<ILikeImpl>;
}

View File

@ -27,6 +27,8 @@ public:
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForNothing() const override { return false; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
size_t getNumberOfArguments() const override

View File

@ -11,7 +11,7 @@ struct NameLike
static constexpr auto name = "like";
};
using LikeImpl = MatchImpl<NameLike, true, false, false>;
using LikeImpl = MatchImpl<NameLike, MatchTraits::Syntax::Like, MatchTraits::Case::Sensitive, MatchTraits::Result::DontNegate>;
using FunctionLike = FunctionsStringSearch<LikeImpl>;
}

View File

@ -6,7 +6,7 @@ namespace DB
{
/// Transforms the [I]LIKE expression into regexp re2. For example, abc%def -> ^abc.*def$
inline String likePatternToRegexp(const String & pattern)
inline String likePatternToRegexp(std::string_view pattern)
{
String res;
res.reserve(pattern.size() * 2);

View File

@ -65,6 +65,8 @@ public:
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
bool useDefaultImplementationForNulls() const override { return false; }
/// map(..., Nothing) -> Map(..., Nothing)
bool useDefaultImplementationForNothing() const override { return false; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override

View File

@ -13,7 +13,7 @@ struct NameMatch
static constexpr auto name = "match";
};
using FunctionMatch = FunctionsStringSearch<MatchImpl<NameMatch, false, false, false>>;
using FunctionMatch = FunctionsStringSearch<MatchImpl<NameMatch, MatchTraits::Syntax::Re2, MatchTraits::Case::Sensitive, MatchTraits::Result::DontNegate>>;
}

View File

@ -12,7 +12,7 @@ struct NameNotILike
static constexpr auto name = "notILike";
};
using NotILikeImpl = MatchImpl<NameNotILike, true, true, true>;
using NotILikeImpl = MatchImpl<NameNotILike, MatchTraits::Syntax::Like, MatchTraits::Case::Insensitive, MatchTraits::Result::Negate>;
using FunctionNotILike = FunctionsStringSearch<NotILikeImpl>;
}

View File

@ -12,7 +12,7 @@ struct NameNotLike
static constexpr auto name = "notLike";
};
using FunctionNotLike = FunctionsStringSearch<MatchImpl<NameNotLike, true, true, false>>;
using FunctionNotLike = FunctionsStringSearch<MatchImpl<NameNotLike, MatchTraits::Syntax::Like, MatchTraits::Case::Sensitive, MatchTraits::Result::Negate>>;
}

View File

@ -60,7 +60,13 @@ void registerFunctionH3GetPentagonIndexes(FunctionFactory &);
void registerFunctionH3Line(FunctionFactory &);
void registerFunctionH3Distance(FunctionFactory &);
void registerFunctionH3HexRing(FunctionFactory &);
void registerFunctionH3GetUnidirectionalEdge(FunctionFactory &);
void registerFunctionH3UnidirectionalEdgeIsValid(FunctionFactory &);
void registerFunctionH3GetOriginIndexFromUnidirectionalEdge(FunctionFactory &);
void registerFunctionH3GetDestinationIndexFromUnidirectionalEdge(FunctionFactory &);
void registerFunctionH3GetIndexesFromUnidirectionalEdge(FunctionFactory &);
void registerFunctionH3GetUnidirectionalEdgesFromHexagon(FunctionFactory &);
void registerFunctionH3GetUnidirectionalEdgeBoundary(FunctionFactory &);
#endif
#if USE_S2_GEOMETRY
@ -134,6 +140,13 @@ void registerFunctionsGeo(FunctionFactory & factory)
registerFunctionH3Line(factory);
registerFunctionH3Distance(factory);
registerFunctionH3HexRing(factory);
registerFunctionH3GetUnidirectionalEdge(factory);
registerFunctionH3UnidirectionalEdgeIsValid(factory);
registerFunctionH3GetOriginIndexFromUnidirectionalEdge(factory);
registerFunctionH3GetDestinationIndexFromUnidirectionalEdge(factory);
registerFunctionH3GetIndexesFromUnidirectionalEdge(factory);
registerFunctionH3GetUnidirectionalEdgesFromHexagon(factory);
registerFunctionH3GetUnidirectionalEdgeBoundary(factory);
#endif
#if USE_S2_GEOMETRY

View File

@ -26,6 +26,8 @@ public:
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForNothing() const override { return false; }
bool isShortCircuit(ShortCircuitSettings & settings, size_t /*number_of_arguments*/) const override
{
settings.enable_lazy_execution_for_first_argument = true;

View File

@ -28,6 +28,7 @@ public:
size_t getNumberOfArguments() const override { return 1; }
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForNothing() const override { return false; }
bool useDefaultImplementationForConstants() const override { return true; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }

View File

@ -30,6 +30,8 @@ public:
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForNothing() const override { return false; }
bool isShortCircuit(ShortCircuitSettings & settings, size_t /*number_of_arguments*/) const override
{
settings.enable_lazy_execution_for_first_argument = false;

View File

@ -52,6 +52,8 @@ public:
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
bool useDefaultImplementationForNulls() const override { return false; }
/// tuple(..., Nothing, ...) -> Tuple(..., Nothing, ...)
bool useDefaultImplementationForNothing() const override { return false; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override

View File

@ -2,6 +2,8 @@
#include <cmath>
#include <cstring>
#include <string>
#include <string_view>
#include <limits>
#include <algorithm>
#include <iterator>
@ -1366,6 +1368,12 @@ inline T parse(const String & s)
return parse<T>(s.data(), s.size());
}
template <typename T>
inline T parse(std::string_view s)
{
return parse<T>(s.data(), s.size());
}
template <typename T>
inline bool tryParse(T & res, const char * data)
{
@ -1378,6 +1386,12 @@ inline bool tryParse(T & res, const String & s)
return tryParse(res, s.data(), s.size());
}
template <typename T>
inline bool tryParse(T & res, std::string_view s)
{
return tryParse(res, s.data(), s.size());
}
/** Skip UTF-8 BOM if it is under cursor.
* As BOM is usually located at start of stream, and buffer size is usually larger than three bytes,

View File

@ -1078,7 +1078,7 @@ void Context::addQueryFactoriesInfo(QueryLogFactories factory_type, const String
if (isGlobalContext())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have query factories info");
auto lock = getLock();
std::lock_guard lock(query_factories_info.mutex);
switch (factory_type)
{

View File

@ -291,6 +291,43 @@ private:
/// Record names of created objects of factories (for testing, etc)
struct QueryFactoriesInfo
{
QueryFactoriesInfo() = default;
QueryFactoriesInfo(const QueryFactoriesInfo & rhs)
{
std::lock_guard<std::mutex> lock(rhs.mutex);
aggregate_functions = rhs.aggregate_functions;
aggregate_function_combinators = rhs.aggregate_function_combinators;
database_engines = rhs.database_engines;
data_type_families = rhs.data_type_families;
dictionaries = rhs.dictionaries;
formats = rhs.formats;
functions = rhs.functions;
storages = rhs.storages;
table_functions = rhs.table_functions;
}
QueryFactoriesInfo(QueryFactoriesInfo && rhs) = delete;
QueryFactoriesInfo & operator=(QueryFactoriesInfo rhs)
{
swap(rhs);
return *this;
}
void swap(QueryFactoriesInfo & rhs)
{
std::swap(aggregate_functions, rhs.aggregate_functions);
std::swap(aggregate_function_combinators, rhs.aggregate_function_combinators);
std::swap(database_engines, rhs.database_engines);
std::swap(data_type_families, rhs.data_type_families);
std::swap(dictionaries, rhs.dictionaries);
std::swap(formats, rhs.formats);
std::swap(functions, rhs.functions);
std::swap(storages, rhs.storages);
std::swap(table_functions, rhs.table_functions);
}
std::unordered_set<std::string> aggregate_functions;
std::unordered_set<std::string> aggregate_function_combinators;
std::unordered_set<std::string> database_engines;
@ -300,9 +337,11 @@ private:
std::unordered_set<std::string> functions;
std::unordered_set<std::string> storages;
std::unordered_set<std::string> table_functions;
mutable std::mutex mutex;
};
/// Needs to be chandged while having const context in factories methods
/// Needs to be changed while having const context in factories methods
mutable QueryFactoriesInfo query_factories_info;
/// TODO: maybe replace with temporary tables?

View File

@ -459,11 +459,26 @@ void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, Context
std::unordered_set<String> group_by_hashes;
if (auto group_by = select_query->groupBy())
{
for (auto & elem : group_by->children)
if (select_query->group_by_with_grouping_sets)
{
auto hash = elem->getTreeHash();
String key = toString(hash.first) + '_' + toString(hash.second);
group_by_hashes.insert(key);
for (auto & set : group_by->children)
{
for (auto & elem : set->children)
{
auto hash = elem->getTreeHash();
String key = toString(hash.first) + '_' + toString(hash.second);
group_by_hashes.insert(key);
}
}
}
else
{
for (auto & elem : group_by->children)
{
auto hash = elem->getTreeHash();
String key = toString(hash.first) + '_' + toString(hash.second);
group_by_hashes.insert(key);
}
}
}

View File

@ -58,6 +58,9 @@ void ASTDictionaryAttributeDeclaration::formatImpl(const FormatSettings & settin
if (hierarchical)
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "HIERARCHICAL";
if (bidirectional)
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "BIDIRECTIONAL";
if (injective)
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "INJECTIVE";

View File

@ -20,6 +20,8 @@ public:
ASTPtr expression;
/// Is attribute mirrored to the parent identifier
bool hierarchical;
/// Is hierarchical attribute bidirectional
bool bidirectional;
/// Flag that shows whether the id->attribute image is injective
bool injective;
/// MongoDB object ID

View File

@ -15,6 +15,7 @@ bool ParserDictionaryAttributeDeclaration::parseImpl(Pos & pos, ASTPtr & node, E
ParserKeyword s_default{"DEFAULT"};
ParserKeyword s_expression{"EXPRESSION"};
ParserKeyword s_hierarchical{"HIERARCHICAL"};
ParserKeyword s_bidirectional{"BIDIRECTIONAL"};
ParserKeyword s_injective{"INJECTIVE"};
ParserKeyword s_is_object_id{"IS_OBJECT_ID"};
ParserLiteral default_parser;
@ -30,6 +31,7 @@ bool ParserDictionaryAttributeDeclaration::parseImpl(Pos & pos, ASTPtr & node, E
ASTPtr default_value;
ASTPtr expression;
bool hierarchical = false;
bool bidirectional = false;
bool injective = false;
bool is_object_id = false;
@ -63,6 +65,12 @@ bool ParserDictionaryAttributeDeclaration::parseImpl(Pos & pos, ASTPtr & node, E
continue;
}
if (!bidirectional && s_bidirectional.ignore(pos, expected))
{
bidirectional = true;
continue;
}
if (!injective && s_injective.ignore(pos, expected))
{
injective = true;
@ -101,6 +109,7 @@ bool ParserDictionaryAttributeDeclaration::parseImpl(Pos & pos, ASTPtr & node, E
}
attribute_declaration->hierarchical = hierarchical;
attribute_declaration->bidirectional = bidirectional;
attribute_declaration->injective = injective;
attribute_declaration->is_object_id = is_object_id;

View File

@ -141,6 +141,16 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write
CurrentMetrics::Increment metric_increment{CurrentMetrics::ReplicatedSend};
{
auto disk = part->volume->getDisk();
UInt64 revision = parse<UInt64>(params.get("disk_revision", "0"));
if (revision)
disk->syncRevision(revision);
revision = disk->getRevision();
if (revision)
response.addCookie({"disk_revision", toString(revision)});
}
if (client_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE)
writeBinary(part->checksums.getTotalSizeOnDisk(), out);
@ -419,6 +429,13 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
{"compress", "false"}
});
if (disk)
{
UInt64 revision = disk->getRevision();
if (revision)
uri.addQueryParameter("disk_revision", toString(revision));
}
Strings capability;
if (try_zero_copy && data_settings->allow_remote_fs_zero_copy_replication)
{
@ -503,6 +520,10 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
if (!disk)
disk = reservation->getDisk();
UInt64 revision = parse<UInt64>(in.getResponseCookie("disk_revision", "0"));
if (revision)
disk->syncRevision(revision);
bool sync = (data_settings->min_compressed_bytes_to_fsync_after_fetch
&& sum_files_size >= data_settings->min_compressed_bytes_to_fsync_after_fetch);

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