mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge master
This commit is contained in:
commit
25b1ed526d
@ -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
|
||||
|
@ -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} \
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
}
|
@ -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
|
||||
|
@ -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:
|
||||
|
||||
|
@ -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}
|
||||
|
||||
|
@ -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 won’t 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 won’t 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 can’t 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 can’t 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:
|
||||
|
||||
|
@ -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**
|
||||
|
||||
|
@ -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`).
|
||||
|
||||
|
@ -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 haven’t 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 haven’t 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:
|
||||
|
@ -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:
|
||||
|
||||
|
@ -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:
|
||||
|
||||
|
@ -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 isn’t already overloaded.
|
||||
ClickHouse Keeper is recommended to replace ZooKeeper for ClickHouse clusters. See the documentation for [ClickHouse Keeper](clickhouse-keeper.md)
|
||||
|
||||
It’s 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.
|
||||
|
||||
|
@ -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>
|
||||
|
@ -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-->
|
@ -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)
|
||||
|
@ -19,3 +19,4 @@
|
||||
#cmakedefine01 ENABLE_CLICKHOUSE_KEEPER
|
||||
#cmakedefine01 ENABLE_CLICKHOUSE_KEEPER_CONVERTER
|
||||
#cmakedefine01 ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER
|
||||
#cmakedefine01 ENABLE_CLICKHOUSE_SU
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
|
@ -1395,8 +1395,11 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
fs::create_directories(format_schema_path);
|
||||
|
||||
/// Check sanity of MergeTreeSettings on server startup
|
||||
global_context->getMergeTreeSettings().sanityCheck(settings);
|
||||
global_context->getReplicatedMergeTreeSettings().sanityCheck(settings);
|
||||
{
|
||||
size_t background_pool_tasks = global_context->getMergeMutateExecutor()->getMaxTasksCount();
|
||||
global_context->getMergeTreeSettings().sanityCheck(background_pool_tasks);
|
||||
global_context->getReplicatedMergeTreeSettings().sanityCheck(background_pool_tasks);
|
||||
}
|
||||
|
||||
/// try set up encryption. There are some errors in config, error will be printed and server wouldn't start.
|
||||
CompressionCodecEncrypted::Configuration::instance().load(config(), "encryption_codecs");
|
||||
|
@ -200,6 +200,6 @@ try
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
std::cerr << DB::getCurrentExceptionMessage(false);
|
||||
std::cerr << DB::getCurrentExceptionMessage(false) << '\n';
|
||||
return 1;
|
||||
}
|
||||
|
3
programs/su/CMakeLists.txt
Normal file
3
programs/su/CMakeLists.txt
Normal file
@ -0,0 +1,3 @@
|
||||
set (CLICKHOUSE_SU_SOURCES clickhouse-su.cpp)
|
||||
set (CLICKHOUSE_SU_LINK PRIVATE dbms)
|
||||
clickhouse_program_add(su)
|
145
programs/su/clickhouse-su.cpp
Normal file
145
programs/su/clickhouse-su.cpp
Normal 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;
|
||||
}
|
@ -159,6 +159,7 @@ enum class AccessType
|
||||
M(SYSTEM_RESTART_REPLICA, "RESTART REPLICA", TABLE, SYSTEM) \
|
||||
M(SYSTEM_RESTORE_REPLICA, "RESTORE REPLICA", TABLE, SYSTEM) \
|
||||
M(SYSTEM_SYNC_DATABASE_REPLICA, "SYNC DATABASE REPLICA", DATABASE, SYSTEM) \
|
||||
M(SYSTEM_SYNC_TRANSACTION_LOG, "SYNC TRANSACTION LOG", GLOBAL, SYSTEM) \
|
||||
M(SYSTEM_FLUSH_DISTRIBUTED, "FLUSH DISTRIBUTED", TABLE, SYSTEM_FLUSH) \
|
||||
M(SYSTEM_FLUSH_LOGS, "FLUSH LOGS", GLOBAL, SYSTEM_FLUSH) \
|
||||
M(SYSTEM_FLUSH, "", GROUP, SYSTEM) \
|
||||
|
@ -87,6 +87,7 @@ add_headers_and_sources(clickhouse_common_io IO/S3)
|
||||
list (REMOVE_ITEM clickhouse_common_io_sources Common/malloc.cpp Common/new_delete.cpp)
|
||||
|
||||
add_headers_and_sources(dbms Disks/IO)
|
||||
add_headers_and_sources(dbms Disks/ObjectStorages)
|
||||
if (TARGET ch_contrib::sqlite)
|
||||
add_headers_and_sources(dbms Databases/SQLite)
|
||||
endif()
|
||||
@ -113,16 +114,16 @@ endif()
|
||||
|
||||
if (TARGET ch_contrib::aws_s3)
|
||||
add_headers_and_sources(dbms Common/S3)
|
||||
add_headers_and_sources(dbms Disks/S3)
|
||||
add_headers_and_sources(dbms Disks/ObjectStorages/S3)
|
||||
endif()
|
||||
|
||||
if (TARGET ch_contrib::azure_sdk)
|
||||
add_headers_and_sources(dbms Disks/AzureBlobStorage)
|
||||
add_headers_and_sources(dbms Disks/ObjectStorages/AzureBlobStorage)
|
||||
endif()
|
||||
|
||||
if (TARGET ch_contrib::hdfs)
|
||||
add_headers_and_sources(dbms Storages/HDFS)
|
||||
add_headers_and_sources(dbms Disks/HDFS)
|
||||
add_headers_and_sources(dbms Disks/ObjectStorages/HDFS)
|
||||
endif()
|
||||
|
||||
add_headers_and_sources(dbms Storages/Cache)
|
||||
|
@ -628,6 +628,7 @@
|
||||
M(657, UNSUPPORTED_MEILISEARCH_TYPE) \
|
||||
M(658, MEILISEARCH_MISSING_SOME_COLUMNS) \
|
||||
M(659, UNKNOWN_STATUS_OF_TRANSACTION) \
|
||||
M(660, HDFS_ERROR) \
|
||||
\
|
||||
M(700, CANNOT_USE_CACHE) \
|
||||
\
|
||||
|
@ -17,20 +17,20 @@ void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration &
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected cache size (`size`) in configuration");
|
||||
|
||||
max_size = config.getUInt64(config_prefix + ".max_size", 0);
|
||||
|
||||
if (max_size == 0)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected non-zero size for cache configuration");
|
||||
|
||||
max_elements = config.getUInt64(config_prefix + ".max_elements", REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS);
|
||||
max_file_segment_size = config.getUInt64(config_prefix + ".max_file_segment_size", REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE);
|
||||
cache_on_write_operations = config.getUInt64(config_prefix + ".cache_on_write_operations", false);
|
||||
do_not_evict_index_and_mark_files = config.getUInt64(config_prefix + ".do_not_evict_index_and_mark_files", true);
|
||||
allow_remove_persistent_cache_by_default = config.getUInt64(config_prefix + ".allow_remove_persistent_cache_by_default", true);
|
||||
|
||||
auto path = config.getString(config_prefix + ".path", "");
|
||||
if (path.empty())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk Cache requires `path` field (cache base path) in config");
|
||||
|
||||
max_elements = config.getUInt64(config_prefix + ".max_elements", REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS);
|
||||
max_file_segment_size = config.getUInt64(config_prefix + ".max_file_segment_size", REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE);
|
||||
|
||||
cache_on_write_operations = config.getUInt64(config_prefix + ".cache_on_write_operations", false);
|
||||
enable_cache_hits_threshold = config.getUInt64(config_prefix + ".enable_cache_hits_threshold", REMOTE_FS_OBJECTS_CACHE_ENABLE_HITS_THRESHOLD);
|
||||
do_not_evict_index_and_mark_files = config.getUInt64(config_prefix + ".do_not_evict_index_and_mark_files", true);
|
||||
allow_remove_persistent_cache_by_default = config.getUInt64(config_prefix + ".allow_remove_persistent_cache_by_default", true);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -17,6 +17,8 @@ struct FileCacheSettings
|
||||
bool do_not_evict_index_and_mark_files = true;
|
||||
bool allow_remove_persistent_cache_by_default = true;
|
||||
|
||||
size_t enable_cache_hits_threshold = REMOTE_FS_OBJECTS_CACHE_ENABLE_HITS_THRESHOLD;
|
||||
|
||||
void loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
|
||||
};
|
||||
|
||||
|
@ -6,6 +6,7 @@ namespace DB
|
||||
|
||||
static constexpr int REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 100 * 1024 * 1024;
|
||||
static constexpr int REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS = 1024 * 1024;
|
||||
static constexpr int REMOTE_FS_OBJECTS_CACHE_ENABLE_HITS_THRESHOLD = 0;
|
||||
|
||||
class IFileCache;
|
||||
using FileCachePtr = std::shared_ptr<IFileCache>;
|
||||
|
@ -67,6 +67,10 @@ FileSegment::FileSegment(
|
||||
write_through_cache_download = true;
|
||||
break;
|
||||
}
|
||||
case (State::SKIP_CACHE):
|
||||
{
|
||||
break;
|
||||
}
|
||||
default:
|
||||
{
|
||||
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Can create cell with either EMPTY, DOWNLOADED, DOWNLOADING state");
|
||||
@ -497,6 +501,14 @@ void FileSegment::complete(std::lock_guard<std::mutex> & cache_lock)
|
||||
|
||||
void FileSegment::completeUnlocked(std::lock_guard<std::mutex> & cache_lock, std::lock_guard<std::mutex> & segment_lock)
|
||||
{
|
||||
bool is_last_holder = cache->isLastFileSegmentHolder(key(), offset(), cache_lock, segment_lock);
|
||||
|
||||
if (is_last_holder && download_state == State::SKIP_CACHE)
|
||||
{
|
||||
cache->remove(key(), offset(), cache_lock, segment_lock);
|
||||
return;
|
||||
}
|
||||
|
||||
if (download_state == State::SKIP_CACHE || is_detached)
|
||||
return;
|
||||
|
||||
@ -514,8 +526,7 @@ void FileSegment::completeUnlocked(std::lock_guard<std::mutex> & cache_lock, std
|
||||
/// Segment state can be changed from DOWNLOADING or EMPTY only if the caller is the
|
||||
/// downloader or the only owner of the segment.
|
||||
|
||||
bool can_update_segment_state = isDownloaderImpl(segment_lock)
|
||||
|| cache->isLastFileSegmentHolder(key(), offset(), cache_lock, segment_lock);
|
||||
bool can_update_segment_state = isDownloaderImpl(segment_lock) || is_last_holder;
|
||||
|
||||
if (can_update_segment_state)
|
||||
download_state = State::PARTIALLY_DOWNLOADED;
|
||||
|
@ -24,6 +24,8 @@ namespace ErrorCodes
|
||||
|
||||
LRUFileCache::LRUFileCache(const String & cache_base_path_, const FileCacheSettings & cache_settings_)
|
||||
: IFileCache(cache_base_path_, cache_settings_)
|
||||
, max_stash_element_size(cache_settings_.max_elements)
|
||||
, enable_cache_hits_threshold(cache_settings_.enable_cache_hits_threshold)
|
||||
, log(&Poco::Logger::get("LRUFileCache"))
|
||||
, allow_remove_persistent_cache_by_default(cache_settings_.allow_remove_persistent_cache_by_default)
|
||||
{
|
||||
@ -362,9 +364,42 @@ LRUFileCache::FileSegmentCell * LRUFileCache::addCell(
|
||||
"Cache already exists for key: `{}`, offset: {}, size: {}.\nCurrent cache structure: {}",
|
||||
key.toString(), offset, size, dumpStructureUnlocked(key, cache_lock));
|
||||
|
||||
auto file_segment = std::make_shared<FileSegment>(offset, size, key, this, state, is_persistent);
|
||||
FileSegmentCell cell(std::move(file_segment), this, cache_lock);
|
||||
auto skip_or_download = [&]() -> FileSegmentPtr
|
||||
{
|
||||
FileSegment::State result_state = state;
|
||||
if (state == FileSegment::State::EMPTY && enable_cache_hits_threshold)
|
||||
{
|
||||
auto record = records.find({key, offset});
|
||||
|
||||
if (record == records.end())
|
||||
{
|
||||
auto queue_iter = stash_queue.add(key, offset, 0, cache_lock);
|
||||
records.insert({{key, offset}, queue_iter});
|
||||
|
||||
if (stash_queue.getElementsNum(cache_lock) > max_stash_element_size)
|
||||
{
|
||||
auto remove_queue_iter = stash_queue.begin();
|
||||
records.erase({remove_queue_iter->key, remove_queue_iter->offset});
|
||||
stash_queue.remove(remove_queue_iter, cache_lock);
|
||||
}
|
||||
|
||||
/// For segments that do not reach the download threshold, we do not download them, but directly read them
|
||||
result_state = FileSegment::State::SKIP_CACHE;
|
||||
}
|
||||
else
|
||||
{
|
||||
auto queue_iter = record->second;
|
||||
queue_iter->hits++;
|
||||
stash_queue.moveToEnd(queue_iter, cache_lock);
|
||||
|
||||
result_state = queue_iter->hits >= enable_cache_hits_threshold ? FileSegment::State::EMPTY : FileSegment::State::SKIP_CACHE;
|
||||
}
|
||||
}
|
||||
|
||||
return std::make_shared<FileSegment>(offset, size, key, this, result_state, is_persistent);
|
||||
};
|
||||
|
||||
FileSegmentCell cell(skip_or_download(), this, cache_lock);
|
||||
auto & offsets = files[key];
|
||||
|
||||
if (offsets.empty())
|
||||
@ -439,7 +474,7 @@ bool LRUFileCache::tryReserve(
|
||||
std::vector<FileSegmentCell *> to_evict;
|
||||
std::vector<FileSegmentCell *> trash;
|
||||
|
||||
for (const auto & [entry_key, entry_offset, entry_size] : queue)
|
||||
for (const auto & [entry_key, entry_offset, entry_size, _] : queue)
|
||||
{
|
||||
if (!is_overflow())
|
||||
break;
|
||||
@ -585,7 +620,7 @@ void LRUFileCache::removeIfReleasable(bool remove_persistent_files)
|
||||
std::vector<FileSegment *> to_remove;
|
||||
for (auto it = queue.begin(); it != queue.end();)
|
||||
{
|
||||
const auto & [key, offset, size] = *it++;
|
||||
const auto & [key, offset, size, _] = *it++;
|
||||
auto * cell = getCell(key, offset, cache_lock);
|
||||
if (!cell)
|
||||
throw Exception(
|
||||
@ -607,6 +642,10 @@ void LRUFileCache::removeIfReleasable(bool remove_persistent_files)
|
||||
}
|
||||
}
|
||||
|
||||
/// Remove all access information.
|
||||
records.clear();
|
||||
stash_queue.removeAll(cache_lock);
|
||||
|
||||
#ifndef NDEBUG
|
||||
assertCacheCorrectness(cache_lock);
|
||||
#endif
|
||||
@ -839,6 +878,7 @@ LRUFileCache::FileSegmentCell::FileSegmentCell(
|
||||
queue_iterator = cache->queue.add(file_segment->key(), file_segment->offset(), file_segment->range().size(), cache_lock);
|
||||
break;
|
||||
}
|
||||
case FileSegment::State::SKIP_CACHE:
|
||||
case FileSegment::State::EMPTY:
|
||||
case FileSegment::State::DOWNLOADING:
|
||||
{
|
||||
@ -855,7 +895,7 @@ LRUFileCache::LRUQueue::Iterator LRUFileCache::LRUQueue::add(
|
||||
const IFileCache::Key & key, size_t offset, size_t size, std::lock_guard<std::mutex> & /* cache_lock */)
|
||||
{
|
||||
#ifndef NDEBUG
|
||||
for (const auto & [entry_key, entry_offset, _] : queue)
|
||||
for (const auto & [entry_key, entry_offset, entry_size, entry_hits] : queue)
|
||||
{
|
||||
if (entry_key == key && entry_offset == offset)
|
||||
throw Exception(
|
||||
@ -875,6 +915,12 @@ void LRUFileCache::LRUQueue::remove(Iterator queue_it, std::lock_guard<std::mute
|
||||
queue.erase(queue_it);
|
||||
}
|
||||
|
||||
void LRUFileCache::LRUQueue::removeAll(std::lock_guard<std::mutex> & /* cache_lock */)
|
||||
{
|
||||
queue.clear();
|
||||
cache_size = 0;
|
||||
}
|
||||
|
||||
void LRUFileCache::LRUQueue::moveToEnd(Iterator queue_it, std::lock_guard<std::mutex> & /* cache_lock */)
|
||||
{
|
||||
queue.splice(queue.end(), queue, queue_it);
|
||||
@ -891,7 +937,7 @@ bool LRUFileCache::LRUQueue::contains(
|
||||
{
|
||||
/// This method is used for assertions in debug mode.
|
||||
/// So we do not care about complexity here.
|
||||
for (const auto & [entry_key, entry_offset, size] : queue)
|
||||
for (const auto & [entry_key, entry_offset, size, _] : queue)
|
||||
{
|
||||
if (key == entry_key && offset == entry_offset)
|
||||
return true;
|
||||
@ -904,7 +950,7 @@ void LRUFileCache::LRUQueue::assertCorrectness(LRUFileCache * cache, std::lock_g
|
||||
[[maybe_unused]] size_t total_size = 0;
|
||||
for (auto it = queue.begin(); it != queue.end();)
|
||||
{
|
||||
auto & [key, offset, size] = *it++;
|
||||
auto & [key, offset, size, _] = *it++;
|
||||
|
||||
auto * cell = cache->getCell(key, offset, cache_lock);
|
||||
if (!cell)
|
||||
@ -926,7 +972,7 @@ void LRUFileCache::LRUQueue::assertCorrectness(LRUFileCache * cache, std::lock_g
|
||||
String LRUFileCache::LRUQueue::toString(std::lock_guard<std::mutex> & /* cache_lock */) const
|
||||
{
|
||||
String result;
|
||||
for (const auto & [key, offset, size] : queue)
|
||||
for (const auto & [key, offset, size, _] : queue)
|
||||
{
|
||||
if (!result.empty())
|
||||
result += ", ";
|
||||
|
@ -7,6 +7,8 @@
|
||||
#include <mutex>
|
||||
#include <unordered_map>
|
||||
#include <unordered_set>
|
||||
#include <boost/functional/hash.hpp>
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <map>
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
@ -51,6 +53,7 @@ private:
|
||||
Key key;
|
||||
size_t offset;
|
||||
size_t size;
|
||||
size_t hits = 0;
|
||||
|
||||
FileKeyAndOffset(const Key & key_, size_t offset_, size_t size_) : key(key_), offset(offset_), size(size_) {}
|
||||
};
|
||||
@ -80,6 +83,8 @@ private:
|
||||
|
||||
Iterator end() { return queue.end(); }
|
||||
|
||||
void removeAll(std::lock_guard<std::mutex> & cache_lock);
|
||||
|
||||
private:
|
||||
std::list<FileKeyAndOffset> queue;
|
||||
size_t cache_size = 0;
|
||||
@ -109,8 +114,26 @@ private:
|
||||
using FileSegmentsByOffset = std::map<size_t, FileSegmentCell>;
|
||||
using CachedFiles = std::unordered_map<Key, FileSegmentsByOffset>;
|
||||
|
||||
using AccessKeyAndOffset = std::pair<Key, size_t>;
|
||||
|
||||
struct KeyAndOffsetHash
|
||||
{
|
||||
std::size_t operator()(const AccessKeyAndOffset & key) const
|
||||
{
|
||||
return std::hash<UInt128>()(key.first) ^ std::hash<UInt64>()(key.second);
|
||||
}
|
||||
};
|
||||
|
||||
using AccessRecord = std::unordered_map<AccessKeyAndOffset, LRUQueue::Iterator, KeyAndOffsetHash>;
|
||||
|
||||
CachedFiles files;
|
||||
LRUQueue queue;
|
||||
|
||||
LRUQueue stash_queue;
|
||||
AccessRecord records;
|
||||
size_t max_stash_element_size;
|
||||
size_t enable_cache_hits_threshold;
|
||||
|
||||
Poco::Logger * log;
|
||||
bool allow_remove_persistent_cache_by_default;
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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
@ -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;
|
||||
|
@ -12,6 +12,7 @@ public:
|
||||
WriteBufferFromNuraftBuffer();
|
||||
|
||||
nuraft::ptr<nuraft::buffer> getBuffer();
|
||||
bool isFinished() const { return finalized; }
|
||||
|
||||
~WriteBufferFromNuraftBuffer() override;
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -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,
|
||||
|
@ -86,6 +86,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
|
||||
M(UInt64, s3_max_connections, 1024, "The maximum number of connections per server.", 0) \
|
||||
M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \
|
||||
M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \
|
||||
M(Bool, enable_s3_requests_logging, false, "Enable very explicit logging of S3 requests. Makes sense for debug only.", 0) \
|
||||
M(UInt64, hdfs_replication, 0, "The actual number of replications can be specified when the hdfs file is created.", 0) \
|
||||
M(Bool, hdfs_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables", 0) \
|
||||
M(Bool, hdfs_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in hdfs engine tables", 0) \
|
||||
|
@ -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});
|
||||
|
@ -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;
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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>
|
||||
|
@ -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>;
|
||||
|
@ -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>
|
||||
|
@ -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>;
|
||||
|
@ -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,
|
||||
|
@ -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.
|
||||
|
@ -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.",
|
||||
|
@ -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"));
|
||||
|
@ -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);
|
||||
|
||||
|
@ -1,18 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <azure/storage/blobs.hpp>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> getAzureBlobContainerClient(
|
||||
const Poco::Util::AbstractConfiguration & config, const String & config_prefix);
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -1,169 +0,0 @@
|
||||
#include <Disks/AzureBlobStorage/DiskAzureBlobStorage.h>
|
||||
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
|
||||
#include <Disks/RemoteDisksCommon.h>
|
||||
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
|
||||
#include <Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/ReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/WriteIndirectBufferFromRemoteFS.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <Common/getRandomASCIIString.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int AZURE_BLOB_STORAGE_ERROR;
|
||||
}
|
||||
|
||||
|
||||
DiskAzureBlobStorageSettings::DiskAzureBlobStorageSettings(
|
||||
UInt64 max_single_part_upload_size_,
|
||||
UInt64 min_bytes_for_seek_,
|
||||
int max_single_read_retries_,
|
||||
int max_single_download_retries_,
|
||||
int thread_pool_size_) :
|
||||
max_single_part_upload_size(max_single_part_upload_size_),
|
||||
min_bytes_for_seek(min_bytes_for_seek_),
|
||||
max_single_read_retries(max_single_read_retries_),
|
||||
max_single_download_retries(max_single_download_retries_),
|
||||
thread_pool_size(thread_pool_size_) {}
|
||||
|
||||
|
||||
DiskAzureBlobStorage::DiskAzureBlobStorage(
|
||||
const String & name_,
|
||||
DiskPtr metadata_disk_,
|
||||
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
|
||||
SettingsPtr settings_,
|
||||
GetDiskSettings settings_getter_) :
|
||||
IDiskRemote(name_, "", metadata_disk_, "DiskAzureBlobStorage", settings_->thread_pool_size),
|
||||
blob_container_client(blob_container_client_),
|
||||
current_settings(std::move(settings_)),
|
||||
settings_getter(settings_getter_) {}
|
||||
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> DiskAzureBlobStorage::readFile(
|
||||
const String & path,
|
||||
const ReadSettings & read_settings,
|
||||
std::optional<size_t>,
|
||||
std::optional<size_t>) const
|
||||
{
|
||||
auto settings = current_settings.get();
|
||||
auto metadata = readMetadata(path);
|
||||
|
||||
LOG_TEST(log, "Read from file by path: {}", backQuote(metadata_disk->getPath() + path));
|
||||
|
||||
auto reader_impl = std::make_unique<ReadBufferFromAzureBlobStorageGather>(
|
||||
blob_container_client, metadata.remote_fs_root_path, metadata.remote_fs_objects,
|
||||
settings->max_single_read_retries, settings->max_single_download_retries, read_settings);
|
||||
|
||||
if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)
|
||||
{
|
||||
auto reader = getThreadPoolReader();
|
||||
return std::make_unique<AsynchronousReadIndirectBufferFromRemoteFS>(reader, read_settings, std::move(reader_impl));
|
||||
}
|
||||
else
|
||||
{
|
||||
auto buf = std::make_unique<ReadIndirectBufferFromRemoteFS>(std::move(reader_impl));
|
||||
return std::make_unique<SeekAvoidingReadBuffer>(std::move(buf), current_settings.get()->min_bytes_for_seek);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> DiskAzureBlobStorage::writeFile(
|
||||
const String & path,
|
||||
size_t buf_size,
|
||||
WriteMode mode,
|
||||
const WriteSettings &)
|
||||
{
|
||||
auto blob_path = path + "_" + getRandomASCIIString(8); /// NOTE: path contains the tmp_* prefix in the blob name
|
||||
|
||||
LOG_TRACE(log, "{} to file by path: {}. AzureBlob Storage path: {}",
|
||||
mode == WriteMode::Rewrite ? "Write" : "Append", backQuote(metadata_disk->getPath() + path), blob_path);
|
||||
|
||||
auto buffer = std::make_unique<WriteBufferFromAzureBlobStorage>(
|
||||
blob_container_client,
|
||||
blob_path,
|
||||
current_settings.get()->max_single_part_upload_size,
|
||||
buf_size);
|
||||
|
||||
auto create_metadata_callback = [this, path, mode, blob_path] (size_t count)
|
||||
{
|
||||
readOrCreateUpdateAndStoreMetadata(path, mode, false, [blob_path, count] (Metadata & metadata) { metadata.addObject(blob_path, count); return true; });
|
||||
};
|
||||
|
||||
return std::make_unique<WriteIndirectBufferFromRemoteFS>(std::move(buffer), std::move(create_metadata_callback), blob_path);
|
||||
}
|
||||
|
||||
|
||||
DiskType DiskAzureBlobStorage::getType() const
|
||||
{
|
||||
return DiskType::AzureBlobStorage;
|
||||
}
|
||||
|
||||
|
||||
bool DiskAzureBlobStorage::isRemote() const
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
bool DiskAzureBlobStorage::supportZeroCopyReplication() const
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
bool DiskAzureBlobStorage::checkUniqueId(const String & id) const
|
||||
{
|
||||
Azure::Storage::Blobs::ListBlobsOptions blobs_list_options;
|
||||
blobs_list_options.Prefix = id;
|
||||
blobs_list_options.PageSizeHint = 1;
|
||||
|
||||
auto blobs_list_response = blob_container_client->ListBlobs(blobs_list_options);
|
||||
auto blobs_list = blobs_list_response.Blobs;
|
||||
|
||||
for (const auto & blob : blobs_list)
|
||||
{
|
||||
if (id == blob.Name)
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
void DiskAzureBlobStorage::removeFromRemoteFS(const std::vector<String> & paths)
|
||||
{
|
||||
for (const auto & path : paths)
|
||||
{
|
||||
try
|
||||
{
|
||||
auto delete_info = blob_container_client->DeleteBlob(path);
|
||||
if (!delete_info.Value.Deleted)
|
||||
throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Failed to delete file in AzureBlob Storage: {}", path);
|
||||
}
|
||||
catch (const Azure::Storage::StorageException & e)
|
||||
{
|
||||
LOG_INFO(log, "Caught an error while deleting file {} : {}", path, e.Message);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void DiskAzureBlobStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String &, const DisksMap &)
|
||||
{
|
||||
auto new_settings = settings_getter(config, "storage_configuration.disks." + name, context);
|
||||
|
||||
current_settings.set(std::move(new_settings));
|
||||
|
||||
if (AsyncExecutor * exec = dynamic_cast<AsyncExecutor*>(&getExecutor()))
|
||||
exec->setMaxThreads(current_settings.get()->thread_pool_size);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -1,86 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <IO/ReadBufferFromAzureBlobStorage.h>
|
||||
#include <IO/WriteBufferFromAzureBlobStorage.h>
|
||||
#include <IO/SeekAvoidingReadBuffer.h>
|
||||
|
||||
#include <azure/identity/managed_identity_credential.hpp>
|
||||
#include <azure/storage/blobs.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct DiskAzureBlobStorageSettings final
|
||||
{
|
||||
DiskAzureBlobStorageSettings(
|
||||
UInt64 max_single_part_upload_size_,
|
||||
UInt64 min_bytes_for_seek_,
|
||||
int max_single_read_retries,
|
||||
int max_single_download_retries,
|
||||
int thread_pool_size_);
|
||||
|
||||
size_t max_single_part_upload_size; /// NOTE: on 32-bit machines it will be at most 4GB, but size_t is also used in BufferBase for offset
|
||||
UInt64 min_bytes_for_seek;
|
||||
size_t max_single_read_retries;
|
||||
size_t max_single_download_retries;
|
||||
size_t thread_pool_size;
|
||||
};
|
||||
|
||||
|
||||
class DiskAzureBlobStorage final : public IDiskRemote
|
||||
{
|
||||
public:
|
||||
|
||||
using SettingsPtr = std::unique_ptr<DiskAzureBlobStorageSettings>;
|
||||
using GetDiskSettings = std::function<SettingsPtr(const Poco::Util::AbstractConfiguration &, const String, ContextPtr)>;
|
||||
|
||||
DiskAzureBlobStorage(
|
||||
const String & name_,
|
||||
DiskPtr metadata_disk_,
|
||||
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
|
||||
SettingsPtr settings_,
|
||||
GetDiskSettings settings_getter_);
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> readFile(
|
||||
const String & path,
|
||||
const ReadSettings & settings,
|
||||
std::optional<size_t> read_hint,
|
||||
std::optional<size_t> file_size) const override;
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> writeFile(
|
||||
const String & path,
|
||||
size_t buf_size,
|
||||
WriteMode mode,
|
||||
const WriteSettings & settings) override;
|
||||
|
||||
DiskType getType() const override;
|
||||
|
||||
bool isRemote() const override;
|
||||
|
||||
bool supportZeroCopyReplication() const override;
|
||||
|
||||
bool checkUniqueId(const String & id) const override;
|
||||
|
||||
void removeFromRemoteFS(const std::vector<String> & paths) override;
|
||||
|
||||
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String &, const DisksMap &) override;
|
||||
|
||||
private:
|
||||
|
||||
/// client used to access the files in the Blob Storage cloud
|
||||
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client;
|
||||
|
||||
MultiVersion<DiskAzureBlobStorageSettings> current_settings;
|
||||
/// Gets disk settings from context.
|
||||
GetDiskSettings settings_getter;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -216,9 +216,9 @@ void DiskDecorator::shutdown()
|
||||
delegate->shutdown();
|
||||
}
|
||||
|
||||
void DiskDecorator::startup()
|
||||
void DiskDecorator::startup(ContextPtr context)
|
||||
{
|
||||
delegate->startup();
|
||||
delegate->startup(context);
|
||||
}
|
||||
|
||||
void DiskDecorator::applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String & config_prefix, const DisksMap & map)
|
||||
|
@ -75,7 +75,7 @@ public:
|
||||
void onFreeze(const String & path) override;
|
||||
SyncGuardPtr getDirectorySyncGuard(const String & path) const override;
|
||||
void shutdown() override;
|
||||
void startup() override;
|
||||
void startup(ContextPtr context) override;
|
||||
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String & config_prefix, const DisksMap & map) override;
|
||||
bool isCached() const override { return delegate->isCached(); }
|
||||
const String & getCacheBasePath() const override { return delegate->getCacheBasePath(); }
|
||||
|
@ -496,7 +496,7 @@ DiskLocal::DiskLocal(
|
||||
disk_checker = std::make_unique<DiskLocalCheckThread>(this, context, local_disk_check_period_ms);
|
||||
}
|
||||
|
||||
void DiskLocal::startup()
|
||||
void DiskLocal::startup(ContextPtr)
|
||||
{
|
||||
try
|
||||
{
|
||||
@ -684,7 +684,7 @@ void registerDiskLocal(DiskFactory & factory)
|
||||
|
||||
std::shared_ptr<IDisk> disk
|
||||
= std::make_shared<DiskLocal>(name, path, keep_free_space_bytes, context, config.getUInt("local_disk_check_period_ms", 0));
|
||||
disk->startup();
|
||||
disk->startup(context);
|
||||
return std::make_shared<DiskRestartProxy>(disk);
|
||||
};
|
||||
factory.registerDiskType("local", creator);
|
||||
|
@ -110,7 +110,7 @@ public:
|
||||
|
||||
bool isBroken() const override { return broken; }
|
||||
|
||||
void startup() override;
|
||||
void startup(ContextPtr) override;
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
|
@ -5,6 +5,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int DEADLOCK_AVOIDED;
|
||||
@ -317,7 +318,7 @@ void DiskRestartProxy::getRemotePathsRecursive(const String & path, std::vector<
|
||||
return DiskDecorator::getRemotePathsRecursive(path, paths_map);
|
||||
}
|
||||
|
||||
void DiskRestartProxy::restart()
|
||||
void DiskRestartProxy::restart(ContextPtr context)
|
||||
{
|
||||
/// Speed up processing unhealthy requests.
|
||||
DiskDecorator::shutdown();
|
||||
@ -340,7 +341,7 @@ void DiskRestartProxy::restart()
|
||||
|
||||
LOG_INFO(log, "Restart lock acquired. Restarting disk {}", DiskDecorator::getName());
|
||||
|
||||
DiskDecorator::startup();
|
||||
DiskDecorator::startup(context);
|
||||
|
||||
LOG_INFO(log, "Disk restarted {}", DiskDecorator::getName());
|
||||
}
|
||||
|
@ -66,7 +66,7 @@ public:
|
||||
std::vector<String> getRemotePaths(const String & path) const override;
|
||||
void getRemotePathsRecursive(const String & path, std::vector<LocalPathWithRemotePaths> & paths_map) override;
|
||||
|
||||
void restart();
|
||||
void restart(ContextPtr context);
|
||||
|
||||
private:
|
||||
friend class RestartAwareReadBuffer;
|
||||
|
@ -9,8 +9,12 @@
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <Disks/IDisk.h>
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
|
||||
#include <Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
|
||||
|
||||
#include <Disks/IO/ReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/WriteIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
|
||||
@ -173,7 +177,7 @@ std::unique_ptr<ReadBufferFromFileBase> DiskWebServer::readFile(const String & p
|
||||
|
||||
if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)
|
||||
{
|
||||
auto reader = IDiskRemote::getThreadPoolReader();
|
||||
auto reader = IObjectStorage::getThreadPoolReader();
|
||||
return std::make_unique<AsynchronousReadIndirectBufferFromRemoteFS>(reader, read_settings, std::move(web_impl), min_bytes_for_seek);
|
||||
}
|
||||
else
|
||||
|
@ -1,10 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <Core/UUID.h>
|
||||
#include <set>
|
||||
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Disks/IDisk.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,143 +0,0 @@
|
||||
#include <Disks/HDFS/DiskHDFS.h>
|
||||
|
||||
#if USE_HDFS
|
||||
|
||||
#include <Disks/DiskLocal.h>
|
||||
#include <Disks/RemoteDisksCommon.h>
|
||||
|
||||
#include <IO/SeekAvoidingReadBuffer.h>
|
||||
#include <Storages/HDFS/WriteBufferFromHDFS.h>
|
||||
#include <Storages/HDFS/HDFSCommon.h>
|
||||
|
||||
#include <Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/ReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/WriteIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
|
||||
|
||||
#include <boost/algorithm/string/predicate.hpp>
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
#include <base/FnTraits.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
DiskHDFS::DiskHDFS(
|
||||
const String & disk_name_,
|
||||
const String & hdfs_root_path_,
|
||||
SettingsPtr settings_,
|
||||
DiskPtr metadata_disk_,
|
||||
const Poco::Util::AbstractConfiguration & config_)
|
||||
: IDiskRemote(disk_name_, hdfs_root_path_, metadata_disk_, "DiskHDFS", settings_->thread_pool_size)
|
||||
, config(config_)
|
||||
, hdfs_builder(createHDFSBuilder(hdfs_root_path_, config))
|
||||
, hdfs_fs(createHDFSFS(hdfs_builder.get()))
|
||||
, settings(std::move(settings_))
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> DiskHDFS::readFile(const String & path, const ReadSettings & read_settings, std::optional<size_t>, std::optional<size_t>) const
|
||||
{
|
||||
auto metadata = readMetadata(path);
|
||||
|
||||
LOG_TEST(log,
|
||||
"Read from file by path: {}. Existing HDFS objects: {}",
|
||||
backQuote(metadata_disk->getPath() + path), metadata.remote_fs_objects.size());
|
||||
|
||||
auto hdfs_impl = std::make_unique<ReadBufferFromHDFSGather>(config, remote_fs_root_path, remote_fs_root_path, metadata.remote_fs_objects, read_settings);
|
||||
auto buf = std::make_unique<ReadIndirectBufferFromRemoteFS>(std::move(hdfs_impl));
|
||||
return std::make_unique<SeekAvoidingReadBuffer>(std::move(buf), settings->min_bytes_for_seek);
|
||||
}
|
||||
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> DiskHDFS::writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings &)
|
||||
{
|
||||
/// Path to store new HDFS object.
|
||||
std::string file_name = getRandomName();
|
||||
std::string hdfs_path = fs::path(remote_fs_root_path) / file_name;
|
||||
|
||||
LOG_TRACE(log, "{} to file by path: {}. HDFS path: {}", mode == WriteMode::Rewrite ? "Write" : "Append",
|
||||
backQuote(metadata_disk->getPath() + path), hdfs_path);
|
||||
|
||||
/// Single O_WRONLY in libhdfs adds O_TRUNC
|
||||
auto hdfs_buffer = std::make_unique<WriteBufferFromHDFS>(hdfs_path,
|
||||
config, settings->replication, buf_size,
|
||||
mode == WriteMode::Rewrite ? O_WRONLY : O_WRONLY | O_APPEND);
|
||||
auto create_metadata_callback = [this, path, mode, file_name] (size_t count)
|
||||
{
|
||||
readOrCreateUpdateAndStoreMetadata(path, mode, false, [file_name, count] (Metadata & metadata) { metadata.addObject(file_name, count); return true; });
|
||||
};
|
||||
|
||||
return std::make_unique<WriteIndirectBufferFromRemoteFS>(std::move(hdfs_buffer), std::move(create_metadata_callback), file_name);
|
||||
}
|
||||
|
||||
void DiskHDFS::removeFromRemoteFS(const std::vector<String> & paths)
|
||||
{
|
||||
for (const auto & hdfs_path : paths)
|
||||
{
|
||||
const size_t begin_of_path = hdfs_path.find('/', hdfs_path.find("//") + 2);
|
||||
|
||||
/// Add path from root to file name
|
||||
int res = hdfsDelete(hdfs_fs.get(), hdfs_path.substr(begin_of_path).c_str(), 0);
|
||||
if (res == -1)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "HDFSDelete failed with path: " + hdfs_path);
|
||||
}
|
||||
}
|
||||
|
||||
bool DiskHDFS::checkUniqueId(const String & hdfs_uri) const
|
||||
{
|
||||
if (!boost::algorithm::starts_with(hdfs_uri, remote_fs_root_path))
|
||||
return false;
|
||||
const size_t begin_of_path = hdfs_uri.find('/', hdfs_uri.find("//") + 2);
|
||||
const String remote_fs_object_path = hdfs_uri.substr(begin_of_path);
|
||||
return (0 == hdfsExists(hdfs_fs.get(), remote_fs_object_path.c_str()));
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
std::unique_ptr<DiskHDFSSettings> getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const Settings & settings)
|
||||
{
|
||||
return std::make_unique<DiskHDFSSettings>(
|
||||
config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024),
|
||||
config.getInt(config_prefix + ".thread_pool_size", 16),
|
||||
config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000),
|
||||
settings.hdfs_replication);
|
||||
}
|
||||
}
|
||||
|
||||
void registerDiskHDFS(DiskFactory & factory)
|
||||
{
|
||||
auto creator = [](const String & name,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const String & config_prefix,
|
||||
ContextPtr context_,
|
||||
const DisksMap & /*map*/) -> DiskPtr
|
||||
{
|
||||
String uri{config.getString(config_prefix + ".endpoint")};
|
||||
checkHDFSURL(uri);
|
||||
|
||||
if (uri.back() != '/')
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "HDFS path must ends with '/', but '{}' doesn't.", uri);
|
||||
|
||||
auto metadata_disk = prepareForLocalMetadata(name, config, config_prefix, context_).second;
|
||||
|
||||
return std::make_shared<DiskHDFS>(
|
||||
name, uri,
|
||||
getSettings(config, config_prefix, context_->getSettingsRef()),
|
||||
metadata_disk, config);
|
||||
};
|
||||
|
||||
factory.registerDiskType("hdfs", creator);
|
||||
}
|
||||
|
||||
}
|
||||
#endif
|
@ -1,84 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_HDFS
|
||||
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <Storages/HDFS/HDFSCommon.h>
|
||||
#include <Core/UUID.h>
|
||||
#include <memory>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct DiskHDFSSettings
|
||||
{
|
||||
size_t min_bytes_for_seek;
|
||||
int thread_pool_size;
|
||||
int objects_chunk_size_to_delete;
|
||||
int replication;
|
||||
|
||||
DiskHDFSSettings(
|
||||
int min_bytes_for_seek_,
|
||||
int thread_pool_size_,
|
||||
int objects_chunk_size_to_delete_,
|
||||
int replication_)
|
||||
: min_bytes_for_seek(min_bytes_for_seek_)
|
||||
, thread_pool_size(thread_pool_size_)
|
||||
, objects_chunk_size_to_delete(objects_chunk_size_to_delete_)
|
||||
, replication(replication_) {}
|
||||
};
|
||||
|
||||
|
||||
/**
|
||||
* Storage for persisting data in HDFS and metadata on the local disk.
|
||||
* Files are represented by file in local filesystem (clickhouse_root/disks/disk_name/path/to/file)
|
||||
* that contains HDFS object key with actual data.
|
||||
*/
|
||||
class DiskHDFS final : public IDiskRemote
|
||||
{
|
||||
public:
|
||||
using SettingsPtr = std::unique_ptr<DiskHDFSSettings>;
|
||||
|
||||
DiskHDFS(
|
||||
const String & disk_name_,
|
||||
const String & hdfs_root_path_,
|
||||
SettingsPtr settings_,
|
||||
DiskPtr metadata_disk_,
|
||||
const Poco::Util::AbstractConfiguration & config_);
|
||||
|
||||
DiskType getType() const override { return DiskType::HDFS; }
|
||||
bool isRemote() const override { return true; }
|
||||
|
||||
bool supportZeroCopyReplication() const override { return true; }
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> readFile(
|
||||
const String & path,
|
||||
const ReadSettings & settings,
|
||||
std::optional<size_t> read_hint,
|
||||
std::optional<size_t> file_size) const override;
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings & settings) override;
|
||||
|
||||
void removeFromRemoteFS(const std::vector<String> & paths) override;
|
||||
|
||||
/// Check file exists and ClickHouse has an access to it
|
||||
/// Overrode in remote disk
|
||||
/// Required for remote disk to ensure that replica has access to data written by other node
|
||||
bool checkUniqueId(const String & hdfs_uri) const override;
|
||||
|
||||
private:
|
||||
String getRandomName() { return toString(UUIDHelpers::generateV4()); }
|
||||
|
||||
const Poco::Util::AbstractConfiguration & config;
|
||||
|
||||
HDFSBuilderWrapper hdfs_builder;
|
||||
HDFSFSPtr hdfs_fs;
|
||||
|
||||
SettingsPtr settings;
|
||||
};
|
||||
|
||||
}
|
||||
#endif
|
@ -10,6 +10,8 @@
|
||||
#include <Disks/DiskType.h>
|
||||
#include <IO/ReadSettings.h>
|
||||
#include <IO/WriteSettings.h>
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <Disks/WriteMode.h>
|
||||
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
@ -48,14 +50,6 @@ class ReadBufferFromFileBase;
|
||||
class WriteBufferFromFileBase;
|
||||
class MMappedFileCache;
|
||||
|
||||
/**
|
||||
* Mode of opening a file for write.
|
||||
*/
|
||||
enum class WriteMode
|
||||
{
|
||||
Rewrite,
|
||||
Append
|
||||
};
|
||||
|
||||
/**
|
||||
* Provide interface for reservation.
|
||||
@ -299,14 +293,14 @@ public:
|
||||
|
||||
virtual bool isReadOnly() const { return false; }
|
||||
|
||||
/// Check if disk is broken. Broken disks will have 0 space and not be used.
|
||||
/// Check if disk is broken. Broken disks will have 0 space and cannot be used.
|
||||
virtual bool isBroken() const { return false; }
|
||||
|
||||
/// Invoked when Global Context is shutdown.
|
||||
virtual void shutdown() {}
|
||||
|
||||
/// Performs action on disk startup.
|
||||
virtual void startup() {}
|
||||
virtual void startup(ContextPtr) {}
|
||||
|
||||
/// Return some uniq string for file, overrode for IDiskRemote
|
||||
/// Required for distinguish different copies of the same part on remote disk
|
||||
|
@ -229,38 +229,6 @@ SeekableReadBufferPtr CachedReadBufferFromFile::getReadBufferForFileSegment(File
|
||||
read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE;
|
||||
return getRemoteFSReadBuffer(file_segment, read_type);
|
||||
}
|
||||
case FileSegment::State::EMPTY:
|
||||
{
|
||||
auto downloader_id = file_segment->getOrSetDownloader();
|
||||
if (downloader_id == file_segment->getCallerId())
|
||||
{
|
||||
if (file_offset_of_buffer_end == file_segment->getDownloadOffset())
|
||||
{
|
||||
read_type = ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE;
|
||||
return getRemoteFSReadBuffer(file_segment, read_type);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// segment{k}
|
||||
/// cache: [______|___________
|
||||
/// ^
|
||||
/// download_offset
|
||||
/// requested_range: [__________]
|
||||
/// ^
|
||||
/// file_offset_of_buffer_end
|
||||
assert(file_offset_of_buffer_end > file_segment->getDownloadOffset());
|
||||
bytes_to_predownload = file_offset_of_buffer_end - file_segment->getDownloadOffset();
|
||||
|
||||
read_type = ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE;
|
||||
return getRemoteFSReadBuffer(file_segment, read_type);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
download_state = file_segment->state();
|
||||
continue;
|
||||
}
|
||||
}
|
||||
case FileSegment::State::DOWNLOADING:
|
||||
{
|
||||
size_t download_offset = file_segment->getDownloadOffset();
|
||||
@ -297,6 +265,7 @@ SeekableReadBufferPtr CachedReadBufferFromFile::getReadBufferForFileSegment(File
|
||||
read_type = ReadType::CACHED;
|
||||
return getCacheReadBuffer(range.left);
|
||||
}
|
||||
case FileSegment::State::EMPTY:
|
||||
case FileSegment::State::PARTIALLY_DOWNLOADED:
|
||||
{
|
||||
if (file_segment->getDownloadOffset() > file_offset_of_buffer_end)
|
||||
@ -508,7 +477,10 @@ bool CachedReadBufferFromFile::completeFileSegmentAndGetNext()
|
||||
|
||||
/// Do not hold pointer to file segment if it is not needed anymore
|
||||
/// so can become releasable and can be evicted from cache.
|
||||
file_segments_holder->file_segments.erase(file_segment_it);
|
||||
/// If the status of filesegment state is SKIP_CACHE, it will not be deleted.
|
||||
/// It will be deleted from the cache when the holder is destructed.
|
||||
if ((*file_segment_it)->state() != FileSegment::State::SKIP_CACHE)
|
||||
file_segments_holder->file_segments.erase(file_segment_it);
|
||||
|
||||
if (current_file_segment_it == file_segments_holder->file_segments.end())
|
||||
return false;
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include "ReadBufferFromRemoteFSGather.h"
|
||||
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <IO/SeekableReadBuffer.h>
|
||||
#include <Disks/IO/ReadBufferFromWebServer.h>
|
||||
|
||||
|
@ -1,9 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/config.h>
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadSettings.h>
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
#include <azure/storage/blobs.hpp>
|
||||
@ -114,7 +114,7 @@ class ReadBufferFromS3Gather final : public ReadBufferFromRemoteFSGather
|
||||
{
|
||||
public:
|
||||
ReadBufferFromS3Gather(
|
||||
std::shared_ptr<Aws::S3::S3Client> client_ptr_,
|
||||
std::shared_ptr<const Aws::S3::S3Client> client_ptr_,
|
||||
const String & bucket_,
|
||||
const String & version_id_,
|
||||
const std::string & common_path_prefix_,
|
||||
@ -132,7 +132,7 @@ public:
|
||||
SeekableReadBufferPtr createImplementationBufferImpl(const String & path, size_t file_size) override;
|
||||
|
||||
private:
|
||||
std::shared_ptr<Aws::S3::S3Client> client_ptr;
|
||||
std::shared_ptr<const Aws::S3::S3Client> client_ptr;
|
||||
String bucket;
|
||||
String version_id;
|
||||
UInt64 max_single_read_retries;
|
||||
@ -146,7 +146,7 @@ class ReadBufferFromAzureBlobStorageGather final : public ReadBufferFromRemoteFS
|
||||
{
|
||||
public:
|
||||
ReadBufferFromAzureBlobStorageGather(
|
||||
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
|
||||
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
|
||||
const std::string & common_path_prefix_,
|
||||
const BlobsPathToSize & blobs_to_read_,
|
||||
size_t max_single_read_retries_,
|
||||
@ -162,7 +162,7 @@ public:
|
||||
SeekableReadBufferPtr createImplementationBufferImpl(const String & path, size_t file_size) override;
|
||||
|
||||
private:
|
||||
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client;
|
||||
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> blob_container_client;
|
||||
size_t max_single_read_retries;
|
||||
size_t max_single_download_retries;
|
||||
};
|
||||
|
@ -2,7 +2,6 @@
|
||||
|
||||
#include <Common/config.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <utility>
|
||||
|
||||
|
||||
|
@ -4,7 +4,6 @@
|
||||
#include <IO/SeekableReadBuffer.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
|
||||
#include <Disks/IDiskRemote.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -36,7 +36,8 @@ WriteIndirectBufferFromRemoteFS::~WriteIndirectBufferFromRemoteFS()
|
||||
void WriteIndirectBufferFromRemoteFS::finalizeImpl()
|
||||
{
|
||||
WriteBufferFromFileDecorator::finalizeImpl();
|
||||
create_metadata_callback(count());
|
||||
if (create_metadata_callback)
|
||||
create_metadata_callback(count());
|
||||
}
|
||||
|
||||
|
||||
|
@ -2,7 +2,6 @@
|
||||
|
||||
#include <Common/config.h>
|
||||
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/WriteBufferFromFileDecorator.h>
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Disks/AzureBlobStorage/AzureBlobStorageAuth.h>
|
||||
#include <Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h>
|
||||
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
|
||||
@ -66,27 +66,27 @@ AzureBlobStorageEndpoint processAzureBlobStorageEndpoint(const Poco::Util::Abstr
|
||||
|
||||
|
||||
template <class T>
|
||||
std::shared_ptr<T> getClientWithConnectionString(const String & connection_str, const String & container_name) = delete;
|
||||
std::unique_ptr<T> getClientWithConnectionString(const String & connection_str, const String & container_name) = delete;
|
||||
|
||||
|
||||
template<>
|
||||
std::shared_ptr<BlobServiceClient> getClientWithConnectionString(
|
||||
std::unique_ptr<BlobServiceClient> getClientWithConnectionString(
|
||||
const String & connection_str, const String & /*container_name*/)
|
||||
{
|
||||
return std::make_shared<BlobServiceClient>(BlobServiceClient::CreateFromConnectionString(connection_str));
|
||||
return std::make_unique<BlobServiceClient>(BlobServiceClient::CreateFromConnectionString(connection_str));
|
||||
}
|
||||
|
||||
|
||||
template<>
|
||||
std::shared_ptr<BlobContainerClient> getClientWithConnectionString(
|
||||
std::unique_ptr<BlobContainerClient> getClientWithConnectionString(
|
||||
const String & connection_str, const String & container_name)
|
||||
{
|
||||
return std::make_shared<BlobContainerClient>(BlobContainerClient::CreateFromConnectionString(connection_str, container_name));
|
||||
return std::make_unique<BlobContainerClient>(BlobContainerClient::CreateFromConnectionString(connection_str, container_name));
|
||||
}
|
||||
|
||||
|
||||
template <class T>
|
||||
std::shared_ptr<T> getAzureBlobStorageClientWithAuth(
|
||||
std::unique_ptr<T> getAzureBlobStorageClientWithAuth(
|
||||
const String & url, const String & container_name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix)
|
||||
{
|
||||
if (config.has(config_prefix + ".connection_string"))
|
||||
@ -101,15 +101,15 @@ std::shared_ptr<T> getAzureBlobStorageClientWithAuth(
|
||||
config.getString(config_prefix + ".account_name"),
|
||||
config.getString(config_prefix + ".account_key")
|
||||
);
|
||||
return std::make_shared<T>(url, storage_shared_key_credential);
|
||||
return std::make_unique<T>(url, storage_shared_key_credential);
|
||||
}
|
||||
|
||||
auto managed_identity_credential = std::make_shared<Azure::Identity::ManagedIdentityCredential>();
|
||||
return std::make_shared<T>(url, managed_identity_credential);
|
||||
return std::make_unique<T>(url, managed_identity_credential);
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<BlobContainerClient> getAzureBlobContainerClient(
|
||||
std::unique_ptr<BlobContainerClient> getAzureBlobContainerClient(
|
||||
const Poco::Util::AbstractConfiguration & config, const String & config_prefix)
|
||||
{
|
||||
auto endpoint = processAzureBlobStorageEndpoint(config, config_prefix);
|
||||
@ -136,10 +136,20 @@ std::shared_ptr<BlobContainerClient> getAzureBlobContainerClient(
|
||||
}
|
||||
}
|
||||
|
||||
return std::make_shared<BlobContainerClient>(
|
||||
return std::make_unique<BlobContainerClient>(
|
||||
blob_service_client->CreateBlobContainer(container_name).Value);
|
||||
}
|
||||
|
||||
std::unique_ptr<AzureObjectStorageSettings> getAzureBlobStorageSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr /*context*/)
|
||||
{
|
||||
return std::make_unique<AzureObjectStorageSettings>(
|
||||
config.getUInt64(config_prefix + ".max_single_part_upload_size", 100 * 1024 * 1024),
|
||||
config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024),
|
||||
config.getInt(config_prefix + ".max_single_read_retries", 3),
|
||||
config.getInt(config_prefix + ".max_single_download_retries", 3)
|
||||
);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -0,0 +1,20 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
|
||||
#include <azure/storage/blobs.hpp>
|
||||
#include <Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
std::unique_ptr<Azure::Storage::Blobs::BlobContainerClient> getAzureBlobContainerClient(
|
||||
const Poco::Util::AbstractConfiguration & config, const String & config_prefix);
|
||||
|
||||
std::unique_ptr<AzureObjectStorageSettings> getAzureBlobStorageSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr /*context*/);
|
||||
|
||||
}
|
||||
|
||||
#endif
|
218
src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp
Normal file
218
src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp
Normal file
@ -0,0 +1,218 @@
|
||||
#include <Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h>
|
||||
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
|
||||
#include <IO/ReadBufferFromAzureBlobStorage.h>
|
||||
#include <IO/WriteBufferFromAzureBlobStorage.h>
|
||||
#include <IO/SeekAvoidingReadBuffer.h>
|
||||
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
|
||||
|
||||
#include <Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int AZURE_BLOB_STORAGE_ERROR;
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
}
|
||||
|
||||
|
||||
AzureObjectStorage::AzureObjectStorage(
|
||||
FileCachePtr && cache_,
|
||||
const String & name_,
|
||||
AzureClientPtr && client_,
|
||||
SettingsPtr && settings_)
|
||||
: IObjectStorage(std::move(cache_))
|
||||
, name(name_)
|
||||
, client(std::move(client_))
|
||||
, settings(std::move(settings_))
|
||||
{
|
||||
}
|
||||
|
||||
bool AzureObjectStorage::exists(const std::string & uri) const
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
|
||||
/// What a shame, no Exists method...
|
||||
Azure::Storage::Blobs::ListBlobsOptions options;
|
||||
options.Prefix = uri;
|
||||
options.PageSizeHint = 1;
|
||||
|
||||
auto blobs_list_response = client_ptr->ListBlobs(options);
|
||||
auto blobs_list = blobs_list_response.Blobs;
|
||||
|
||||
for (const auto & blob : blobs_list)
|
||||
{
|
||||
if (uri == blob.Name)
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
std::unique_ptr<SeekableReadBuffer> AzureObjectStorage::readObject( /// NOLINT
|
||||
const std::string & path,
|
||||
const ReadSettings & read_settings,
|
||||
std::optional<size_t>,
|
||||
std::optional<size_t>) const
|
||||
{
|
||||
auto settings_ptr = settings.get();
|
||||
|
||||
return std::make_unique<ReadBufferFromAzureBlobStorage>(
|
||||
client.get(), path, settings_ptr->max_single_read_retries,
|
||||
settings_ptr->max_single_download_retries, read_settings.remote_fs_buffer_size);
|
||||
}
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> AzureObjectStorage::readObjects( /// NOLINT
|
||||
const std::string & common_path_prefix,
|
||||
const BlobsPathToSize & blobs_to_read,
|
||||
const ReadSettings & read_settings,
|
||||
std::optional<size_t>,
|
||||
std::optional<size_t>) const
|
||||
{
|
||||
auto settings_ptr = settings.get();
|
||||
auto reader_impl = std::make_unique<ReadBufferFromAzureBlobStorageGather>(
|
||||
client.get(), common_path_prefix, blobs_to_read,
|
||||
settings_ptr->max_single_read_retries, settings_ptr->max_single_download_retries, read_settings);
|
||||
|
||||
if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)
|
||||
{
|
||||
auto reader = getThreadPoolReader();
|
||||
return std::make_unique<AsynchronousReadIndirectBufferFromRemoteFS>(reader, read_settings, std::move(reader_impl));
|
||||
}
|
||||
else
|
||||
{
|
||||
auto buf = std::make_unique<ReadIndirectBufferFromRemoteFS>(std::move(reader_impl));
|
||||
return std::make_unique<SeekAvoidingReadBuffer>(std::move(buf), settings_ptr->min_bytes_for_seek);
|
||||
}
|
||||
}
|
||||
|
||||
/// Open the file for write and return WriteBufferFromFileBase object.
|
||||
std::unique_ptr<WriteBufferFromFileBase> AzureObjectStorage::writeObject( /// NOLINT
|
||||
const std::string & path,
|
||||
WriteMode mode,
|
||||
std::optional<ObjectAttributes>,
|
||||
FinalizeCallback && finalize_callback,
|
||||
size_t buf_size,
|
||||
const WriteSettings &)
|
||||
{
|
||||
if (mode != WriteMode::Rewrite)
|
||||
throw Exception("Azure storage doesn't support append", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
|
||||
auto buffer = std::make_unique<WriteBufferFromAzureBlobStorage>(
|
||||
client.get(),
|
||||
path,
|
||||
settings.get()->max_single_part_upload_size,
|
||||
buf_size);
|
||||
|
||||
return std::make_unique<WriteIndirectBufferFromRemoteFS>(std::move(buffer), std::move(finalize_callback), path);
|
||||
}
|
||||
|
||||
void AzureObjectStorage::listPrefix(const std::string & path, BlobsPathToSize & children) const
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
|
||||
Azure::Storage::Blobs::ListBlobsOptions blobs_list_options;
|
||||
blobs_list_options.Prefix = path;
|
||||
|
||||
auto blobs_list_response = client_ptr->ListBlobs(blobs_list_options);
|
||||
auto blobs_list = blobs_list_response.Blobs;
|
||||
|
||||
for (const auto & blob : blobs_list)
|
||||
children.emplace_back(blob.Name, blob.BlobSize);
|
||||
}
|
||||
|
||||
/// Remove file. Throws exception if file doesn't exists or it's a directory.
|
||||
void AzureObjectStorage::removeObject(const std::string & path)
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
auto delete_info = client_ptr->DeleteBlob(path);
|
||||
if (!delete_info.Value.Deleted)
|
||||
throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Failed to delete file in AzureBlob Storage: {}", path);
|
||||
}
|
||||
|
||||
void AzureObjectStorage::removeObjects(const std::vector<std::string> & paths)
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
for (const auto & path : paths)
|
||||
{
|
||||
auto delete_info = client_ptr->DeleteBlob(path);
|
||||
if (!delete_info.Value.Deleted)
|
||||
throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Failed to delete file in AzureBlob Storage: {}", path);
|
||||
}
|
||||
}
|
||||
|
||||
void AzureObjectStorage::removeObjectIfExists(const std::string & path)
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
auto delete_info = client_ptr->DeleteBlob(path);
|
||||
}
|
||||
|
||||
void AzureObjectStorage::removeObjectsIfExist(const std::vector<std::string> & paths)
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
for (const auto & path : paths)
|
||||
auto delete_info = client_ptr->DeleteBlob(path);
|
||||
}
|
||||
|
||||
|
||||
ObjectMetadata AzureObjectStorage::getObjectMetadata(const std::string & path) const
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
auto blob_client = client_ptr->GetBlobClient(path);
|
||||
auto properties = blob_client.GetProperties().Value;
|
||||
ObjectMetadata result;
|
||||
result.size_bytes = properties.BlobSize;
|
||||
if (!properties.Metadata.empty())
|
||||
{
|
||||
result.attributes.emplace();
|
||||
for (const auto & [key, value] : properties.Metadata)
|
||||
(*result.attributes)[key] = value;
|
||||
}
|
||||
result.last_modified.emplace(properties.LastModified.time_since_epoch().count());
|
||||
return result;
|
||||
}
|
||||
|
||||
void AzureObjectStorage::copyObject( /// NOLINT
|
||||
const std::string & object_from,
|
||||
const std::string & object_to,
|
||||
std::optional<ObjectAttributes> object_to_attributes)
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
auto dest_blob_client = client_ptr->GetBlobClient(object_to);
|
||||
auto source_blob_client = client_ptr->GetBlobClient(object_from);
|
||||
Azure::Storage::Blobs::CopyBlobFromUriOptions copy_options;
|
||||
if (object_to_attributes.has_value())
|
||||
{
|
||||
for (const auto & [key, value] : *object_to_attributes)
|
||||
copy_options.Metadata[key] = value;
|
||||
}
|
||||
|
||||
dest_blob_client.CopyFromUri(source_blob_client.GetUrl(), copy_options);
|
||||
}
|
||||
|
||||
void AzureObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context)
|
||||
{
|
||||
auto new_settings = getAzureBlobStorageSettings(config, config_prefix, context);
|
||||
settings.set(std::move(new_settings));
|
||||
|
||||
/// We don't update client
|
||||
}
|
||||
|
||||
|
||||
std::unique_ptr<IObjectStorage> AzureObjectStorage::cloneObjectStorage(const std::string &, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context)
|
||||
{
|
||||
return std::make_unique<AzureObjectStorage>(
|
||||
nullptr,
|
||||
name,
|
||||
getAzureBlobContainerClient(config, config_prefix),
|
||||
getAzureBlobStorageSettings(config, config_prefix, context)
|
||||
);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#endif
|
113
src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h
Normal file
113
src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h
Normal file
@ -0,0 +1,113 @@
|
||||
#pragma once
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageCommon.h>
|
||||
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
|
||||
#include <Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/ReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/WriteIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <Common/getRandomASCIIString.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct AzureObjectStorageSettings
|
||||
{
|
||||
AzureObjectStorageSettings(
|
||||
uint64_t max_single_part_upload_size_,
|
||||
uint64_t min_bytes_for_seek_,
|
||||
int max_single_read_retries_,
|
||||
int max_single_download_retries_)
|
||||
: max_single_part_upload_size(max_single_part_upload_size_)
|
||||
, min_bytes_for_seek(min_bytes_for_seek_)
|
||||
, max_single_read_retries(max_single_read_retries_)
|
||||
, max_single_download_retries(max_single_download_retries_)
|
||||
{
|
||||
}
|
||||
|
||||
size_t max_single_part_upload_size; /// NOTE: on 32-bit machines it will be at most 4GB, but size_t is also used in BufferBase for offset
|
||||
uint64_t min_bytes_for_seek;
|
||||
size_t max_single_read_retries;
|
||||
size_t max_single_download_retries;
|
||||
};
|
||||
|
||||
using AzureClient = Azure::Storage::Blobs::BlobContainerClient;
|
||||
using AzureClientPtr = std::unique_ptr<Azure::Storage::Blobs::BlobContainerClient>;
|
||||
|
||||
class AzureObjectStorage : public IObjectStorage
|
||||
{
|
||||
public:
|
||||
|
||||
using SettingsPtr = std::unique_ptr<AzureObjectStorageSettings>;
|
||||
|
||||
AzureObjectStorage(
|
||||
FileCachePtr && cache_,
|
||||
const String & name_,
|
||||
AzureClientPtr && client_,
|
||||
SettingsPtr && settings_);
|
||||
|
||||
bool exists(const std::string & uri) const override;
|
||||
|
||||
std::unique_ptr<SeekableReadBuffer> readObject( /// NOLINT
|
||||
const std::string & path,
|
||||
const ReadSettings & read_settings = ReadSettings{},
|
||||
std::optional<size_t> read_hint = {},
|
||||
std::optional<size_t> file_size = {}) const override;
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> readObjects( /// NOLINT
|
||||
const std::string & common_path_prefix,
|
||||
const BlobsPathToSize & blobs_to_read,
|
||||
const ReadSettings & read_settings = ReadSettings{},
|
||||
std::optional<size_t> read_hint = {},
|
||||
std::optional<size_t> file_size = {}) const override;
|
||||
|
||||
/// Open the file for write and return WriteBufferFromFileBase object.
|
||||
std::unique_ptr<WriteBufferFromFileBase> writeObject( /// NOLINT
|
||||
const std::string & path,
|
||||
WriteMode mode,
|
||||
std::optional<ObjectAttributes> attributes = {},
|
||||
FinalizeCallback && finalize_callback = {},
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
const WriteSettings & write_settings = {}) override;
|
||||
|
||||
void listPrefix(const std::string & path, BlobsPathToSize & children) const override;
|
||||
/// Remove file. Throws exception if file doesn't exists or it's a directory.
|
||||
void removeObject(const std::string & path) override;
|
||||
|
||||
void removeObjects(const std::vector<std::string> & paths) override;
|
||||
|
||||
void removeObjectIfExists(const std::string & path) override;
|
||||
|
||||
void removeObjectsIfExist(const std::vector<std::string> & paths) override;
|
||||
|
||||
ObjectMetadata getObjectMetadata(const std::string & path) const override;
|
||||
|
||||
void copyObject( /// NOLINT
|
||||
const std::string & object_from,
|
||||
const std::string & object_to,
|
||||
std::optional<ObjectAttributes> object_to_attributes = {}) override;
|
||||
|
||||
void shutdown() override {}
|
||||
|
||||
void startup() override {}
|
||||
|
||||
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) override;
|
||||
|
||||
String getObjectsNamespace() const override { return ""; }
|
||||
|
||||
std::unique_ptr<IObjectStorage> cloneObjectStorage(const std::string & new_namespace, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) override;
|
||||
|
||||
private:
|
||||
const String name;
|
||||
/// client used to access the files in the Blob Storage cloud
|
||||
MultiVersion<Azure::Storage::Blobs::BlobContainerClient> client;
|
||||
MultiVersion<AzureObjectStorageSettings> settings;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -5,11 +5,12 @@
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
|
||||
#include <Disks/DiskRestartProxy.h>
|
||||
#include <Disks/RemoteDisksCommon.h>
|
||||
#include <Disks/AzureBlobStorage/DiskAzureBlobStorage.h>
|
||||
#include <Disks/AzureBlobStorage/AzureBlobStorageAuth.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageCommon.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorage.h>
|
||||
|
||||
#include <Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h>
|
||||
#include <Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -19,18 +20,19 @@ namespace ErrorCodes
|
||||
extern const int PATH_ACCESS_DENIED;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
constexpr char test_file[] = "test.txt";
|
||||
constexpr char test_str[] = "test";
|
||||
constexpr size_t test_str_size = 4;
|
||||
|
||||
|
||||
void checkWriteAccess(IDisk & disk)
|
||||
{
|
||||
auto file = disk.writeFile(test_file, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite);
|
||||
file->write(test_str, test_str_size);
|
||||
}
|
||||
|
||||
|
||||
void checkReadAccess(IDisk & disk)
|
||||
{
|
||||
auto file = disk.readFile(test_file);
|
||||
@ -40,7 +42,6 @@ void checkReadAccess(IDisk & disk)
|
||||
throw Exception("No read access to disk", ErrorCodes::PATH_ACCESS_DENIED);
|
||||
}
|
||||
|
||||
|
||||
void checkReadWithOffset(IDisk & disk)
|
||||
{
|
||||
auto file = disk.readFile(test_file);
|
||||
@ -53,25 +54,13 @@ void checkReadWithOffset(IDisk & disk)
|
||||
throw Exception("Failed to read file with offset", ErrorCodes::PATH_ACCESS_DENIED);
|
||||
}
|
||||
|
||||
|
||||
void checkRemoveAccess(IDisk & disk)
|
||||
{
|
||||
disk.removeFile(test_file);
|
||||
}
|
||||
|
||||
|
||||
std::unique_ptr<DiskAzureBlobStorageSettings> getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr /*context*/)
|
||||
{
|
||||
return std::make_unique<DiskAzureBlobStorageSettings>(
|
||||
config.getUInt64(config_prefix + ".max_single_part_upload_size", 100 * 1024 * 1024),
|
||||
config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024),
|
||||
config.getInt(config_prefix + ".max_single_read_retries", 3),
|
||||
config.getInt(config_prefix + ".max_single_download_retries", 3),
|
||||
config.getInt(config_prefix + ".thread_pool_size", 16)
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
void registerDiskAzureBlobStorage(DiskFactory & factory)
|
||||
{
|
||||
auto creator = [](
|
||||
@ -83,12 +72,25 @@ void registerDiskAzureBlobStorage(DiskFactory & factory)
|
||||
{
|
||||
auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context);
|
||||
|
||||
std::shared_ptr<IDisk> azure_blob_storage_disk = std::make_shared<DiskAzureBlobStorage>(
|
||||
/// FIXME Cache currently unsupported :(
|
||||
ObjectStoragePtr azure_object_storage = std::make_unique<AzureObjectStorage>(
|
||||
nullptr,
|
||||
name,
|
||||
metadata_disk,
|
||||
getAzureBlobContainerClient(config, config_prefix),
|
||||
getSettings(config, config_prefix, context),
|
||||
getSettings
|
||||
getAzureBlobStorageSettings(config, config_prefix, context));
|
||||
|
||||
uint64_t copy_thread_pool_size = config.getUInt(config_prefix + ".thread_pool_size", 16);
|
||||
bool send_metadata = config.getBool(config_prefix + ".send_metadata", false);
|
||||
|
||||
std::shared_ptr<IDisk> azure_blob_storage_disk = std::make_shared<DiskObjectStorage>(
|
||||
name,
|
||||
/* no namespaces */"",
|
||||
"DiskAzureBlobStorage",
|
||||
metadata_disk,
|
||||
std::move(azure_object_storage),
|
||||
DiskType::AzureBlobStorage,
|
||||
send_metadata,
|
||||
copy_thread_pool_size
|
||||
);
|
||||
|
||||
if (!config.getBool(config_prefix + ".skip_access_check", false))
|
||||
@ -99,7 +101,15 @@ void registerDiskAzureBlobStorage(DiskFactory & factory)
|
||||
checkRemoveAccess(*azure_blob_storage_disk);
|
||||
}
|
||||
|
||||
azure_blob_storage_disk->startup();
|
||||
#ifdef NDEBUG
|
||||
bool use_cache = true;
|
||||
#else
|
||||
/// Current cache implementation lead to allocations in destructor of
|
||||
/// read buffer.
|
||||
bool use_cache = false;
|
||||
#endif
|
||||
|
||||
azure_blob_storage_disk->startup(context);
|
||||
|
||||
return std::make_shared<DiskRestartProxy>(azure_blob_storage_disk);
|
||||
};
|
678
src/Disks/ObjectStorages/DiskObjectStorage.cpp
Normal file
678
src/Disks/ObjectStorages/DiskObjectStorage.cpp
Normal file
@ -0,0 +1,678 @@
|
||||
#include <Disks/ObjectStorages/DiskObjectStorage.h>
|
||||
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/createHardLink.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/checkStackSize.h>
|
||||
#include <Common/getRandomASCIIString.h>
|
||||
#include <boost/algorithm/string.hpp>
|
||||
#include <Common/filesystemHelpers.h>
|
||||
#include <Disks/IO/ThreadPoolRemoteFSReader.h>
|
||||
#include <Common/FileCache.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageMetadataHelper.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INCORRECT_DISK_INDEX;
|
||||
extern const int UNKNOWN_FORMAT;
|
||||
extern const int FILE_ALREADY_EXISTS;
|
||||
extern const int FILE_DOESNT_EXIST;
|
||||
extern const int BAD_FILE_TYPE;
|
||||
}
|
||||
|
||||
static String revisionToString(UInt64 revision)
|
||||
{
|
||||
return std::bitset<64>(revision).to_string();
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/// Runs tasks asynchronously using thread pool.
|
||||
class AsyncThreadPoolExecutor : public Executor
|
||||
{
|
||||
public:
|
||||
AsyncThreadPoolExecutor(const String & name_, int thread_pool_size)
|
||||
: name(name_)
|
||||
, pool(ThreadPool(thread_pool_size)) {}
|
||||
|
||||
std::future<void> execute(std::function<void()> task) override
|
||||
{
|
||||
auto promise = std::make_shared<std::promise<void>>();
|
||||
pool.scheduleOrThrowOnError(
|
||||
[promise, task]()
|
||||
{
|
||||
try
|
||||
{
|
||||
task();
|
||||
promise->set_value();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException("Failed to run async task");
|
||||
|
||||
try
|
||||
{
|
||||
promise->set_exception(std::current_exception());
|
||||
}
|
||||
catch (...) {}
|
||||
}
|
||||
});
|
||||
|
||||
return promise->get_future();
|
||||
}
|
||||
|
||||
void setMaxThreads(size_t threads)
|
||||
{
|
||||
pool.setMaxThreads(threads);
|
||||
}
|
||||
|
||||
private:
|
||||
String name;
|
||||
ThreadPool pool;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
DiskObjectStorage::DiskObjectStorage(
|
||||
const String & name_,
|
||||
const String & remote_fs_root_path_,
|
||||
const String & log_name,
|
||||
DiskPtr metadata_disk_,
|
||||
ObjectStoragePtr && object_storage_,
|
||||
DiskType disk_type_,
|
||||
bool send_metadata_,
|
||||
uint64_t thread_pool_size)
|
||||
: IDisk(std::make_unique<AsyncThreadPoolExecutor>(log_name, thread_pool_size))
|
||||
, name(name_)
|
||||
, remote_fs_root_path(remote_fs_root_path_)
|
||||
, log (&Poco::Logger::get(log_name))
|
||||
, metadata_disk(metadata_disk_)
|
||||
, disk_type(disk_type_)
|
||||
, object_storage(std::move(object_storage_))
|
||||
, send_metadata(send_metadata_)
|
||||
, metadata_helper(std::make_unique<DiskObjectStorageMetadataHelper>(this, ReadSettings{}))
|
||||
{}
|
||||
|
||||
DiskObjectStorage::Metadata DiskObjectStorage::readMetadataUnlocked(const String & path, std::shared_lock<std::shared_mutex> &) const
|
||||
{
|
||||
return Metadata::readMetadata(remote_fs_root_path, metadata_disk, path);
|
||||
}
|
||||
|
||||
|
||||
DiskObjectStorage::Metadata DiskObjectStorage::readMetadata(const String & path) const
|
||||
{
|
||||
std::shared_lock lock(metadata_mutex);
|
||||
return readMetadataUnlocked(path, lock);
|
||||
}
|
||||
|
||||
DiskObjectStorage::Metadata DiskObjectStorage::readUpdateAndStoreMetadata(const String & path, bool sync, DiskObjectStorage::MetadataUpdater updater)
|
||||
{
|
||||
std::unique_lock lock(metadata_mutex);
|
||||
return Metadata::readUpdateAndStoreMetadata(remote_fs_root_path, metadata_disk, path, sync, updater);
|
||||
}
|
||||
|
||||
|
||||
DiskObjectStorage::Metadata DiskObjectStorage::readUpdateStoreMetadataAndRemove(const String & path, bool sync, DiskObjectStorage::MetadataUpdater updater)
|
||||
{
|
||||
std::unique_lock lock(metadata_mutex);
|
||||
return Metadata::readUpdateStoreMetadataAndRemove(remote_fs_root_path, metadata_disk, path, sync, updater);
|
||||
}
|
||||
|
||||
DiskObjectStorage::Metadata DiskObjectStorage::readOrCreateUpdateAndStoreMetadata(const String & path, WriteMode mode, bool sync, DiskObjectStorage::MetadataUpdater updater)
|
||||
{
|
||||
if (mode == WriteMode::Rewrite || !metadata_disk->exists(path))
|
||||
{
|
||||
std::unique_lock lock(metadata_mutex);
|
||||
return Metadata::createUpdateAndStoreMetadata(remote_fs_root_path, metadata_disk, path, sync, updater);
|
||||
}
|
||||
else
|
||||
{
|
||||
return Metadata::readUpdateAndStoreMetadata(remote_fs_root_path, metadata_disk, path, sync, updater);
|
||||
}
|
||||
}
|
||||
|
||||
DiskObjectStorage::Metadata DiskObjectStorage::createAndStoreMetadata(const String & path, bool sync)
|
||||
{
|
||||
return Metadata::createAndStoreMetadata(remote_fs_root_path, metadata_disk, path, sync);
|
||||
}
|
||||
|
||||
DiskObjectStorage::Metadata DiskObjectStorage::createUpdateAndStoreMetadata(const String & path, bool sync, DiskObjectStorage::MetadataUpdater updater)
|
||||
{
|
||||
return Metadata::createUpdateAndStoreMetadata(remote_fs_root_path, metadata_disk, path, sync, updater);
|
||||
}
|
||||
|
||||
std::vector<String> DiskObjectStorage::getRemotePaths(const String & local_path) const
|
||||
{
|
||||
auto metadata = readMetadata(local_path);
|
||||
|
||||
std::vector<String> remote_paths;
|
||||
for (const auto & [remote_path, _] : metadata.remote_fs_objects)
|
||||
remote_paths.push_back(fs::path(metadata.remote_fs_root_path) / remote_path);
|
||||
|
||||
return remote_paths;
|
||||
|
||||
}
|
||||
|
||||
void DiskObjectStorage::getRemotePathsRecursive(const String & local_path, std::vector<LocalPathWithRemotePaths> & paths_map)
|
||||
{
|
||||
/// Protect against concurrent delition of files (for example because of a merge).
|
||||
if (metadata_disk->isFile(local_path))
|
||||
{
|
||||
try
|
||||
{
|
||||
paths_map.emplace_back(local_path, getRemotePaths(local_path));
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
if (e.code() == ErrorCodes::FILE_DOESNT_EXIST)
|
||||
return;
|
||||
throw;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
DiskDirectoryIteratorPtr it;
|
||||
try
|
||||
{
|
||||
it = iterateDirectory(local_path);
|
||||
}
|
||||
catch (const fs::filesystem_error & e)
|
||||
{
|
||||
if (e.code() == std::errc::no_such_file_or_directory)
|
||||
return;
|
||||
throw;
|
||||
}
|
||||
|
||||
for (; it->isValid(); it->next())
|
||||
DiskObjectStorage::getRemotePathsRecursive(fs::path(local_path) / it->name(), paths_map);
|
||||
}
|
||||
}
|
||||
|
||||
bool DiskObjectStorage::exists(const String & path) const
|
||||
{
|
||||
return metadata_disk->exists(path);
|
||||
}
|
||||
|
||||
|
||||
bool DiskObjectStorage::isFile(const String & path) const
|
||||
{
|
||||
return metadata_disk->isFile(path);
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorage::createFile(const String & path)
|
||||
{
|
||||
createAndStoreMetadata(path, false);
|
||||
}
|
||||
|
||||
size_t DiskObjectStorage::getFileSize(const String & path) const
|
||||
{
|
||||
return readMetadata(path).total_size;
|
||||
}
|
||||
|
||||
void DiskObjectStorage::moveFile(const String & from_path, const String & to_path, bool should_send_metadata)
|
||||
{
|
||||
if (exists(to_path))
|
||||
throw Exception("File already exists: " + to_path, ErrorCodes::FILE_ALREADY_EXISTS);
|
||||
|
||||
if (should_send_metadata)
|
||||
{
|
||||
auto revision = metadata_helper->revision_counter + 1;
|
||||
metadata_helper->revision_counter += 1;
|
||||
|
||||
const ObjectAttributes object_metadata {
|
||||
{"from_path", from_path},
|
||||
{"to_path", to_path}
|
||||
};
|
||||
metadata_helper->createFileOperationObject("rename", revision, object_metadata);
|
||||
}
|
||||
|
||||
metadata_disk->moveFile(from_path, to_path);
|
||||
}
|
||||
|
||||
void DiskObjectStorage::moveFile(const String & from_path, const String & to_path)
|
||||
{
|
||||
moveFile(from_path, to_path, send_metadata);
|
||||
}
|
||||
|
||||
void DiskObjectStorage::replaceFile(const String & from_path, const String & to_path)
|
||||
{
|
||||
if (exists(to_path))
|
||||
{
|
||||
const String tmp_path = to_path + ".old";
|
||||
moveFile(to_path, tmp_path);
|
||||
moveFile(from_path, to_path);
|
||||
removeFile(tmp_path);
|
||||
}
|
||||
else
|
||||
moveFile(from_path, to_path);
|
||||
}
|
||||
|
||||
void DiskObjectStorage::removeSharedFile(const String & path, bool delete_metadata_only)
|
||||
{
|
||||
std::vector<String> paths_to_remove;
|
||||
removeMetadata(path, paths_to_remove);
|
||||
|
||||
if (!delete_metadata_only)
|
||||
removeFromRemoteFS(paths_to_remove);
|
||||
}
|
||||
|
||||
void DiskObjectStorage::removeFromRemoteFS(const std::vector<String> & paths)
|
||||
{
|
||||
object_storage->removeObjects(paths);
|
||||
}
|
||||
|
||||
UInt32 DiskObjectStorage::getRefCount(const String & path) const
|
||||
{
|
||||
return readMetadata(path).ref_count;
|
||||
}
|
||||
|
||||
std::unordered_map<String, String> DiskObjectStorage::getSerializedMetadata(const std::vector<String> & file_paths) const
|
||||
{
|
||||
std::unordered_map<String, String> metadatas;
|
||||
|
||||
std::shared_lock lock(metadata_mutex);
|
||||
|
||||
for (const auto & path : file_paths)
|
||||
{
|
||||
DiskObjectStorage::Metadata metadata = readMetadataUnlocked(path, lock);
|
||||
metadata.ref_count = 0;
|
||||
metadatas[path] = metadata.serializeToString();
|
||||
}
|
||||
|
||||
return metadatas;
|
||||
}
|
||||
|
||||
String DiskObjectStorage::getUniqueId(const String & path) const
|
||||
{
|
||||
LOG_TRACE(log, "Remote path: {}, Path: {}", remote_fs_root_path, path);
|
||||
auto metadata = readMetadata(path);
|
||||
String id;
|
||||
if (!metadata.remote_fs_objects.empty())
|
||||
id = metadata.remote_fs_root_path + metadata.remote_fs_objects[0].relative_path;
|
||||
return id;
|
||||
}
|
||||
|
||||
bool DiskObjectStorage::checkObjectExists(const String & path) const
|
||||
{
|
||||
if (!path.starts_with(remote_fs_root_path))
|
||||
return false;
|
||||
|
||||
return object_storage->exists(path);
|
||||
}
|
||||
|
||||
bool DiskObjectStorage::checkUniqueId(const String & id) const
|
||||
{
|
||||
return checkObjectExists(id);
|
||||
}
|
||||
|
||||
void DiskObjectStorage::createHardLink(const String & src_path, const String & dst_path, bool should_send_metadata)
|
||||
{
|
||||
readUpdateAndStoreMetadata(src_path, false, [](Metadata & metadata) { metadata.ref_count++; return true; });
|
||||
|
||||
if (should_send_metadata && !dst_path.starts_with("shadow/"))
|
||||
{
|
||||
auto revision = metadata_helper->revision_counter + 1;
|
||||
metadata_helper->revision_counter += 1;
|
||||
const ObjectAttributes object_metadata {
|
||||
{"src_path", src_path},
|
||||
{"dst_path", dst_path}
|
||||
};
|
||||
metadata_helper->createFileOperationObject("hardlink", revision, object_metadata);
|
||||
}
|
||||
|
||||
/// Create FS hardlink to metadata file.
|
||||
metadata_disk->createHardLink(src_path, dst_path);
|
||||
}
|
||||
|
||||
void DiskObjectStorage::createHardLink(const String & src_path, const String & dst_path)
|
||||
{
|
||||
createHardLink(src_path, dst_path, send_metadata);
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorage::setReadOnly(const String & path)
|
||||
{
|
||||
/// We should store read only flag inside metadata file (instead of using FS flag),
|
||||
/// because we modify metadata file when create hard-links from it.
|
||||
readUpdateAndStoreMetadata(path, false, [](Metadata & metadata) { metadata.read_only = true; return true; });
|
||||
}
|
||||
|
||||
|
||||
bool DiskObjectStorage::isDirectory(const String & path) const
|
||||
{
|
||||
return metadata_disk->isDirectory(path);
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorage::createDirectory(const String & path)
|
||||
{
|
||||
metadata_disk->createDirectory(path);
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorage::createDirectories(const String & path)
|
||||
{
|
||||
metadata_disk->createDirectories(path);
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorage::clearDirectory(const String & path)
|
||||
{
|
||||
for (auto it = iterateDirectory(path); it->isValid(); it->next())
|
||||
if (isFile(it->path()))
|
||||
removeFile(it->path());
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorage::removeDirectory(const String & path)
|
||||
{
|
||||
metadata_disk->removeDirectory(path);
|
||||
}
|
||||
|
||||
|
||||
DiskDirectoryIteratorPtr DiskObjectStorage::iterateDirectory(const String & path)
|
||||
{
|
||||
return metadata_disk->iterateDirectory(path);
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorage::listFiles(const String & path, std::vector<String> & file_names)
|
||||
{
|
||||
for (auto it = iterateDirectory(path); it->isValid(); it->next())
|
||||
file_names.push_back(it->name());
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorage::setLastModified(const String & path, const Poco::Timestamp & timestamp)
|
||||
{
|
||||
metadata_disk->setLastModified(path, timestamp);
|
||||
}
|
||||
|
||||
|
||||
Poco::Timestamp DiskObjectStorage::getLastModified(const String & path)
|
||||
{
|
||||
return metadata_disk->getLastModified(path);
|
||||
}
|
||||
|
||||
void DiskObjectStorage::removeMetadata(const String & path, std::vector<String> & paths_to_remove)
|
||||
{
|
||||
LOG_TRACE(log, "Remove file by path: {}", backQuote(metadata_disk->getPath() + path));
|
||||
|
||||
if (!metadata_disk->exists(path))
|
||||
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Metadata path '{}' doesn't exist", path);
|
||||
|
||||
if (!metadata_disk->isFile(path))
|
||||
throw Exception(ErrorCodes::BAD_FILE_TYPE, "Path '{}' is not a regular file", path);
|
||||
|
||||
try
|
||||
{
|
||||
auto metadata_updater = [&paths_to_remove, this] (Metadata & metadata)
|
||||
{
|
||||
if (metadata.ref_count == 0)
|
||||
{
|
||||
for (const auto & [remote_fs_object_path, _] : metadata.remote_fs_objects)
|
||||
{
|
||||
String object_path = fs::path(remote_fs_root_path) / remote_fs_object_path;
|
||||
paths_to_remove.push_back(object_path);
|
||||
object_storage->removeFromCache(object_path);
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
else /// In other case decrement number of references, save metadata and delete hardlink.
|
||||
{
|
||||
--metadata.ref_count;
|
||||
}
|
||||
|
||||
return true;
|
||||
};
|
||||
|
||||
readUpdateStoreMetadataAndRemove(path, false, metadata_updater);
|
||||
/// If there is no references - delete content from remote FS.
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
/// If it's impossible to read meta - just remove it from FS.
|
||||
if (e.code() == ErrorCodes::UNKNOWN_FORMAT)
|
||||
{
|
||||
LOG_WARNING(log,
|
||||
"Metadata file {} can't be read by reason: {}. Removing it forcibly.",
|
||||
backQuote(path), e.nested() ? e.nested()->message() : e.message());
|
||||
metadata_disk->removeFile(path);
|
||||
}
|
||||
else
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorage::removeMetadataRecursive(const String & path, std::unordered_map<String, std::vector<String>> & paths_to_remove)
|
||||
{
|
||||
checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks.
|
||||
|
||||
if (metadata_disk->isFile(path))
|
||||
{
|
||||
removeMetadata(path, paths_to_remove[path]);
|
||||
}
|
||||
else
|
||||
{
|
||||
for (auto it = iterateDirectory(path); it->isValid(); it->next())
|
||||
removeMetadataRecursive(it->path(), paths_to_remove);
|
||||
|
||||
metadata_disk->removeDirectory(path);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorage::shutdown()
|
||||
{
|
||||
LOG_INFO(log, "Shutting down disk {}", name);
|
||||
object_storage->shutdown();
|
||||
LOG_INFO(log, "Disk {} shut down", name);
|
||||
}
|
||||
|
||||
void DiskObjectStorage::startup(ContextPtr context)
|
||||
{
|
||||
|
||||
LOG_INFO(log, "Starting up disk {}", name);
|
||||
object_storage->startup();
|
||||
|
||||
restoreMetadataIfNeeded(context->getConfigRef(), "storage_configuration.disks." + name, context);
|
||||
|
||||
LOG_INFO(log, "Disk {} started up", name);
|
||||
}
|
||||
|
||||
ReservationPtr DiskObjectStorage::reserve(UInt64 bytes)
|
||||
{
|
||||
if (!tryReserve(bytes))
|
||||
return {};
|
||||
|
||||
return std::make_unique<DiskObjectStorageReservation>(std::static_pointer_cast<DiskObjectStorage>(shared_from_this()), bytes);
|
||||
}
|
||||
|
||||
void DiskObjectStorage::removeSharedFileIfExists(const String & path, bool delete_metadata_only)
|
||||
{
|
||||
std::vector<String> paths_to_remove;
|
||||
if (metadata_disk->exists(path))
|
||||
{
|
||||
removeMetadata(path, paths_to_remove);
|
||||
if (!delete_metadata_only)
|
||||
removeFromRemoteFS(paths_to_remove);
|
||||
}
|
||||
}
|
||||
|
||||
void DiskObjectStorage::removeSharedRecursive(const String & path, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only)
|
||||
{
|
||||
std::unordered_map<String, std::vector<String>> paths_to_remove;
|
||||
removeMetadataRecursive(path, paths_to_remove);
|
||||
|
||||
if (!keep_all_batch_data)
|
||||
{
|
||||
std::vector<String> remove_from_remote;
|
||||
for (auto && [local_path, remote_paths] : paths_to_remove)
|
||||
{
|
||||
if (!file_names_remove_metadata_only.contains(fs::path(local_path).filename()))
|
||||
{
|
||||
remove_from_remote.insert(remove_from_remote.end(), remote_paths.begin(), remote_paths.end());
|
||||
}
|
||||
}
|
||||
removeFromRemoteFS(remove_from_remote);
|
||||
}
|
||||
}
|
||||
|
||||
std::optional<UInt64> DiskObjectStorage::tryReserve(UInt64 bytes)
|
||||
{
|
||||
std::lock_guard lock(reservation_mutex);
|
||||
|
||||
auto available_space = getAvailableSpace();
|
||||
UInt64 unreserved_space = available_space - std::min(available_space, reserved_bytes);
|
||||
|
||||
if (bytes == 0)
|
||||
{
|
||||
LOG_TRACE(log, "Reserving 0 bytes on remote_fs disk {}", backQuote(name));
|
||||
++reservation_count;
|
||||
return {unreserved_space};
|
||||
}
|
||||
|
||||
if (unreserved_space >= bytes)
|
||||
{
|
||||
LOG_TRACE(log, "Reserving {} on disk {}, having unreserved {}.",
|
||||
ReadableSize(bytes), backQuote(name), ReadableSize(unreserved_space));
|
||||
++reservation_count;
|
||||
reserved_bytes += bytes;
|
||||
return {unreserved_space - bytes};
|
||||
}
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> DiskObjectStorage::readFile(
|
||||
const String & path,
|
||||
const ReadSettings & settings,
|
||||
std::optional<size_t> read_hint,
|
||||
std::optional<size_t> file_size) const
|
||||
{
|
||||
auto metadata = readMetadata(path);
|
||||
return object_storage->readObjects(remote_fs_root_path, metadata.remote_fs_objects, settings, read_hint, file_size);
|
||||
}
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> DiskObjectStorage::writeFile(
|
||||
const String & path,
|
||||
size_t buf_size,
|
||||
WriteMode mode,
|
||||
const WriteSettings & settings)
|
||||
{
|
||||
auto blob_name = getRandomASCIIString();
|
||||
|
||||
std::optional<ObjectAttributes> object_attributes;
|
||||
if (send_metadata)
|
||||
{
|
||||
auto revision = metadata_helper->revision_counter + 1;
|
||||
metadata_helper->revision_counter++;
|
||||
object_attributes = {
|
||||
{"path", path}
|
||||
};
|
||||
blob_name = "r" + revisionToString(revision) + "-file-" + blob_name;
|
||||
}
|
||||
|
||||
auto create_metadata_callback = [this, path, blob_name, mode] (size_t count)
|
||||
{
|
||||
readOrCreateUpdateAndStoreMetadata(path, mode, false,
|
||||
[blob_name, count] (DiskObjectStorage::Metadata & metadata) { metadata.addObject(blob_name, count); return true; });
|
||||
};
|
||||
|
||||
/// We always use mode Rewrite because we simulate append using metadata and different files
|
||||
return object_storage->writeObject(
|
||||
fs::path(remote_fs_root_path) / blob_name, WriteMode::Rewrite, object_attributes,
|
||||
std::move(create_metadata_callback),
|
||||
buf_size, settings);
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context_, const String &, const DisksMap &)
|
||||
{
|
||||
const auto config_prefix = "storage_configuration.disks." + name;
|
||||
object_storage->applyNewSettings(config, config_prefix, context_);
|
||||
|
||||
if (AsyncThreadPoolExecutor * exec = dynamic_cast<AsyncThreadPoolExecutor *>(&getExecutor()))
|
||||
exec->setMaxThreads(config.getInt(config_prefix + ".thread_pool_size", 16));
|
||||
}
|
||||
|
||||
void DiskObjectStorage::restoreMetadataIfNeeded(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context)
|
||||
{
|
||||
if (send_metadata)
|
||||
{
|
||||
metadata_helper->restore(config, config_prefix, context);
|
||||
|
||||
if (metadata_helper->readSchemaVersion(object_storage.get(), remote_fs_root_path) < DiskObjectStorageMetadataHelper::RESTORABLE_SCHEMA_VERSION)
|
||||
metadata_helper->migrateToRestorableSchema();
|
||||
|
||||
metadata_helper->findLastRevision();
|
||||
}
|
||||
}
|
||||
|
||||
void DiskObjectStorage::syncRevision(UInt64 revision)
|
||||
{
|
||||
metadata_helper->syncRevision(revision);
|
||||
}
|
||||
|
||||
UInt64 DiskObjectStorage::getRevision() const
|
||||
{
|
||||
return metadata_helper->getRevision();
|
||||
}
|
||||
|
||||
|
||||
DiskPtr DiskObjectStorageReservation::getDisk(size_t i) const
|
||||
{
|
||||
if (i != 0)
|
||||
throw Exception("Can't use i != 0 with single disk reservation", ErrorCodes::INCORRECT_DISK_INDEX);
|
||||
return disk;
|
||||
}
|
||||
|
||||
void DiskObjectStorageReservation::update(UInt64 new_size)
|
||||
{
|
||||
std::lock_guard lock(disk->reservation_mutex);
|
||||
disk->reserved_bytes -= size;
|
||||
size = new_size;
|
||||
disk->reserved_bytes += size;
|
||||
}
|
||||
|
||||
DiskObjectStorageReservation::~DiskObjectStorageReservation()
|
||||
{
|
||||
try
|
||||
{
|
||||
std::lock_guard lock(disk->reservation_mutex);
|
||||
if (disk->reserved_bytes < size)
|
||||
{
|
||||
disk->reserved_bytes = 0;
|
||||
LOG_ERROR(disk->log, "Unbalanced reservations size for disk '{}'.", disk->getName());
|
||||
}
|
||||
else
|
||||
{
|
||||
disk->reserved_bytes -= size;
|
||||
}
|
||||
|
||||
if (disk->reservation_count == 0)
|
||||
LOG_ERROR(disk->log, "Unbalanced reservation count for disk '{}'.", disk->getName());
|
||||
else
|
||||
--disk->reservation_count;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
232
src/Disks/ObjectStorages/DiskObjectStorage.h
Normal file
232
src/Disks/ObjectStorages/DiskObjectStorage.h
Normal file
@ -0,0 +1,232 @@
|
||||
#pragma once
|
||||
|
||||
#include <Disks/IDisk.h>
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageMetadataHelper.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageMetadata.h>
|
||||
#include <re2/re2.h>
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric DiskSpaceReservedForMerge;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Disk build on top of IObjectStorage. Use additional disk (local for example)
|
||||
/// for metadata storage. Metadata is a small files with mapping from local paths to
|
||||
/// objects in object storage, like:
|
||||
/// "/var/lib/clickhouse/data/db/table/all_0_0_0/columns.txt" -> /xxxxxxxxxxxxxxxxxxxx
|
||||
/// -> /yyyyyyyyyyyyyyyyyyyy
|
||||
class DiskObjectStorage : public IDisk
|
||||
{
|
||||
|
||||
friend class DiskObjectStorageReservation;
|
||||
friend class DiskObjectStorageMetadataHelper;
|
||||
|
||||
public:
|
||||
DiskObjectStorage(
|
||||
const String & name_,
|
||||
const String & remote_fs_root_path_,
|
||||
const String & log_name,
|
||||
DiskPtr metadata_disk_,
|
||||
ObjectStoragePtr && object_storage_,
|
||||
DiskType disk_type_,
|
||||
bool send_metadata_,
|
||||
uint64_t thread_pool_size);
|
||||
|
||||
DiskType getType() const override { return disk_type; }
|
||||
|
||||
bool supportZeroCopyReplication() const override { return true; }
|
||||
|
||||
bool supportParallelWrite() const override { return true; }
|
||||
|
||||
using Metadata = DiskObjectStorageMetadata;
|
||||
using MetadataUpdater = std::function<bool(Metadata & metadata)>;
|
||||
|
||||
const String & getName() const override { return name; }
|
||||
|
||||
const String & getPath() const override { return metadata_disk->getPath(); }
|
||||
|
||||
std::vector<String> getRemotePaths(const String & local_path) const override;
|
||||
|
||||
void getRemotePathsRecursive(const String & local_path, std::vector<LocalPathWithRemotePaths> & paths_map) override;
|
||||
|
||||
std::string getCacheBasePath() const override
|
||||
{
|
||||
return object_storage->getCacheBasePath();
|
||||
}
|
||||
|
||||
/// Methods for working with metadata. For some operations (like hardlink
|
||||
/// creation) metadata can be updated concurrently from multiple threads
|
||||
/// (file actually rewritten on disk). So additional RW lock is required for
|
||||
/// metadata read and write, but not for create new metadata.
|
||||
Metadata readMetadata(const String & path) const;
|
||||
Metadata readMetadataUnlocked(const String & path, std::shared_lock<std::shared_mutex> &) const;
|
||||
Metadata readUpdateAndStoreMetadata(const String & path, bool sync, MetadataUpdater updater);
|
||||
Metadata readUpdateStoreMetadataAndRemove(const String & path, bool sync, MetadataUpdater updater);
|
||||
|
||||
Metadata readOrCreateUpdateAndStoreMetadata(const String & path, WriteMode mode, bool sync, MetadataUpdater updater);
|
||||
|
||||
Metadata createAndStoreMetadata(const String & path, bool sync);
|
||||
Metadata createUpdateAndStoreMetadata(const String & path, bool sync, MetadataUpdater updater);
|
||||
|
||||
UInt64 getTotalSpace() const override { return std::numeric_limits<UInt64>::max(); }
|
||||
|
||||
UInt64 getAvailableSpace() const override { return std::numeric_limits<UInt64>::max(); }
|
||||
|
||||
UInt64 getUnreservedSpace() const override { return std::numeric_limits<UInt64>::max(); }
|
||||
|
||||
UInt64 getKeepingFreeSpace() const override { return 0; }
|
||||
|
||||
bool exists(const String & path) const override;
|
||||
|
||||
bool isFile(const String & path) const override;
|
||||
|
||||
void createFile(const String & path) override;
|
||||
|
||||
size_t getFileSize(const String & path) const override;
|
||||
|
||||
void moveFile(const String & from_path, const String & to_path) override;
|
||||
|
||||
void moveFile(const String & from_path, const String & to_path, bool should_send_metadata);
|
||||
|
||||
void replaceFile(const String & from_path, const String & to_path) override;
|
||||
|
||||
void removeFile(const String & path) override { removeSharedFile(path, false); }
|
||||
|
||||
void removeFileIfExists(const String & path) override { removeSharedFileIfExists(path, false); }
|
||||
|
||||
void removeRecursive(const String & path) override { removeSharedRecursive(path, false, {}); }
|
||||
|
||||
void removeSharedFile(const String & path, bool delete_metadata_only) override;
|
||||
|
||||
void removeSharedFileIfExists(const String & path, bool delete_metadata_only) override;
|
||||
|
||||
void removeSharedRecursive(const String & path, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override;
|
||||
|
||||
void removeFromRemoteFS(const std::vector<String> & paths);
|
||||
|
||||
DiskPtr getMetadataDiskIfExistsOrSelf() override { return metadata_disk; }
|
||||
|
||||
UInt32 getRefCount(const String & path) const override;
|
||||
|
||||
/// Return metadata for each file path. Also, before serialization reset
|
||||
/// ref_count for each metadata to zero. This function used only for remote
|
||||
/// fetches/sends in replicated engines. That's why we reset ref_count to zero.
|
||||
std::unordered_map<String, String> getSerializedMetadata(const std::vector<String> & file_paths) const override;
|
||||
|
||||
String getUniqueId(const String & path) const override;
|
||||
|
||||
bool checkObjectExists(const String & path) const;
|
||||
bool checkUniqueId(const String & id) const override;
|
||||
|
||||
void createHardLink(const String & src_path, const String & dst_path) override;
|
||||
void createHardLink(const String & src_path, const String & dst_path, bool should_send_metadata);
|
||||
|
||||
void listFiles(const String & path, std::vector<String> & file_names) override;
|
||||
|
||||
void setReadOnly(const String & path) override;
|
||||
|
||||
bool isDirectory(const String & path) const override;
|
||||
|
||||
void createDirectory(const String & path) override;
|
||||
|
||||
void createDirectories(const String & path) override;
|
||||
|
||||
void clearDirectory(const String & path) override;
|
||||
|
||||
void moveDirectory(const String & from_path, const String & to_path) override { moveFile(from_path, to_path); }
|
||||
|
||||
void removeDirectory(const String & path) override;
|
||||
|
||||
DiskDirectoryIteratorPtr iterateDirectory(const String & path) override;
|
||||
|
||||
void setLastModified(const String & path, const Poco::Timestamp & timestamp) override;
|
||||
|
||||
Poco::Timestamp getLastModified(const String & path) override;
|
||||
|
||||
bool isRemote() const override { return true; }
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
void startup(ContextPtr context) override;
|
||||
|
||||
ReservationPtr reserve(UInt64 bytes) override;
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> readFile(
|
||||
const String & path,
|
||||
const ReadSettings & settings,
|
||||
std::optional<size_t> read_hint,
|
||||
std::optional<size_t> file_size) const override;
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> writeFile(
|
||||
const String & path,
|
||||
size_t buf_size,
|
||||
WriteMode mode,
|
||||
const WriteSettings & settings) override;
|
||||
|
||||
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context_, const String &, const DisksMap &) override;
|
||||
|
||||
void restoreMetadataIfNeeded(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context);
|
||||
|
||||
void onFreeze(const String & path) override;
|
||||
|
||||
void syncRevision(UInt64 revision) override;
|
||||
|
||||
UInt64 getRevision() const override;
|
||||
private:
|
||||
const String name;
|
||||
const String remote_fs_root_path;
|
||||
Poco::Logger * log;
|
||||
DiskPtr metadata_disk;
|
||||
|
||||
const DiskType disk_type;
|
||||
ObjectStoragePtr object_storage;
|
||||
|
||||
UInt64 reserved_bytes = 0;
|
||||
UInt64 reservation_count = 0;
|
||||
std::mutex reservation_mutex;
|
||||
|
||||
mutable std::shared_mutex metadata_mutex;
|
||||
void removeMetadata(const String & path, std::vector<String> & paths_to_remove);
|
||||
|
||||
void removeMetadataRecursive(const String & path, std::unordered_map<String, std::vector<String>> & paths_to_remove);
|
||||
|
||||
std::optional<UInt64> tryReserve(UInt64 bytes);
|
||||
|
||||
bool send_metadata;
|
||||
|
||||
std::unique_ptr<DiskObjectStorageMetadataHelper> metadata_helper;
|
||||
};
|
||||
|
||||
class DiskObjectStorageReservation final : public IReservation
|
||||
{
|
||||
public:
|
||||
DiskObjectStorageReservation(const std::shared_ptr<DiskObjectStorage> & disk_, UInt64 size_)
|
||||
: disk(disk_)
|
||||
, size(size_)
|
||||
, metric_increment(CurrentMetrics::DiskSpaceReservedForMerge, size_)
|
||||
{}
|
||||
|
||||
UInt64 getSize() const override { return size; }
|
||||
|
||||
UInt64 getUnreservedSpace() const override { return unreserved_space; }
|
||||
|
||||
DiskPtr getDisk(size_t i) const override;
|
||||
|
||||
Disks getDisks() const override { return {disk}; }
|
||||
|
||||
void update(UInt64 new_size) override;
|
||||
|
||||
~DiskObjectStorageReservation() override;
|
||||
|
||||
private:
|
||||
std::shared_ptr<DiskObjectStorage> disk;
|
||||
UInt64 size;
|
||||
UInt64 unreserved_space;
|
||||
CurrentMetrics::Increment metric_increment;
|
||||
};
|
||||
|
||||
}
|
@ -1,4 +1,4 @@
|
||||
#include <Disks/RemoteDisksCommon.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageCommon.h>
|
||||
#include <Common/getRandomASCIIString.h>
|
||||
#include <Disks/DiskLocal.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
@ -6,6 +6,12 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
std::shared_ptr<DiskCacheWrapper> wrapWithCache(
|
||||
static String getDiskMetadataPath(
|
||||
const String & name,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
@ -2,11 +2,13 @@
|
||||
|
||||
#include <random>
|
||||
#include <utility>
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
#include <Disks/IDisk.h>
|
||||
#include <Common/getRandomASCIIString.h>
|
||||
|
||||
#include <Disks/IDisk.h>
|
||||
#include <Disks/DiskCacheWrapper.h>
|
||||
|
||||
namespace DB
|
||||
{
|
208
src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp
Normal file
208
src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp
Normal file
@ -0,0 +1,208 @@
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageMetadata.h>
|
||||
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNKNOWN_FORMAT;
|
||||
extern const int PATH_ACCESS_DENIED;
|
||||
extern const int MEMORY_LIMIT_EXCEEDED;
|
||||
}
|
||||
|
||||
DiskObjectStorageMetadata DiskObjectStorageMetadata::readMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_)
|
||||
{
|
||||
DiskObjectStorageMetadata result(remote_fs_root_path_, metadata_disk_, metadata_file_path_);
|
||||
result.load();
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
DiskObjectStorageMetadata DiskObjectStorageMetadata::createAndStoreMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync)
|
||||
{
|
||||
DiskObjectStorageMetadata result(remote_fs_root_path_, metadata_disk_, metadata_file_path_);
|
||||
result.save(sync);
|
||||
return result;
|
||||
}
|
||||
|
||||
DiskObjectStorageMetadata DiskObjectStorageMetadata::readUpdateAndStoreMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, DiskObjectStorageMetadataUpdater updater)
|
||||
{
|
||||
DiskObjectStorageMetadata result(remote_fs_root_path_, metadata_disk_, metadata_file_path_);
|
||||
result.load();
|
||||
if (updater(result))
|
||||
result.save(sync);
|
||||
return result;
|
||||
}
|
||||
|
||||
DiskObjectStorageMetadata DiskObjectStorageMetadata::createUpdateAndStoreMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, DiskObjectStorageMetadataUpdater updater)
|
||||
{
|
||||
DiskObjectStorageMetadata result(remote_fs_root_path_, metadata_disk_, metadata_file_path_);
|
||||
updater(result);
|
||||
result.save(sync);
|
||||
return result;
|
||||
}
|
||||
|
||||
DiskObjectStorageMetadata DiskObjectStorageMetadata::readUpdateStoreMetadataAndRemove(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, DiskObjectStorageMetadataUpdater updater)
|
||||
{
|
||||
DiskObjectStorageMetadata result(remote_fs_root_path_, metadata_disk_, metadata_file_path_);
|
||||
result.load();
|
||||
if (updater(result))
|
||||
result.save(sync);
|
||||
metadata_disk_->removeFile(metadata_file_path_);
|
||||
|
||||
return result;
|
||||
|
||||
}
|
||||
|
||||
DiskObjectStorageMetadata DiskObjectStorageMetadata::createAndStoreMetadataIfNotExists(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, bool overwrite)
|
||||
{
|
||||
if (overwrite || !metadata_disk_->exists(metadata_file_path_))
|
||||
{
|
||||
return createAndStoreMetadata(remote_fs_root_path_, metadata_disk_, metadata_file_path_, sync);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto result = readMetadata(remote_fs_root_path_, metadata_disk_, metadata_file_path_);
|
||||
if (result.read_only)
|
||||
throw Exception("File is read-only: " + metadata_file_path_, ErrorCodes::PATH_ACCESS_DENIED);
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadata::load()
|
||||
{
|
||||
try
|
||||
{
|
||||
const ReadSettings read_settings;
|
||||
auto buf = metadata_disk->readFile(metadata_file_path, read_settings, 1024); /* reasonable buffer size for small file */
|
||||
|
||||
UInt32 version;
|
||||
readIntText(version, *buf);
|
||||
|
||||
if (version < VERSION_ABSOLUTE_PATHS || version > VERSION_READ_ONLY_FLAG)
|
||||
throw Exception(
|
||||
ErrorCodes::UNKNOWN_FORMAT,
|
||||
"Unknown metadata file version. Path: {}. Version: {}. Maximum expected version: {}",
|
||||
metadata_disk->getPath() + metadata_file_path, toString(version), toString(VERSION_READ_ONLY_FLAG));
|
||||
|
||||
assertChar('\n', *buf);
|
||||
|
||||
UInt32 remote_fs_objects_count;
|
||||
readIntText(remote_fs_objects_count, *buf);
|
||||
assertChar('\t', *buf);
|
||||
readIntText(total_size, *buf);
|
||||
assertChar('\n', *buf);
|
||||
remote_fs_objects.resize(remote_fs_objects_count);
|
||||
|
||||
for (size_t i = 0; i < remote_fs_objects_count; ++i)
|
||||
{
|
||||
String remote_fs_object_path;
|
||||
size_t remote_fs_object_size;
|
||||
readIntText(remote_fs_object_size, *buf);
|
||||
assertChar('\t', *buf);
|
||||
readEscapedString(remote_fs_object_path, *buf);
|
||||
if (version == VERSION_ABSOLUTE_PATHS)
|
||||
{
|
||||
if (!remote_fs_object_path.starts_with(remote_fs_root_path))
|
||||
throw Exception(ErrorCodes::UNKNOWN_FORMAT,
|
||||
"Path in metadata does not correspond to root path. Path: {}, root path: {}, disk path: {}",
|
||||
remote_fs_object_path, remote_fs_root_path, metadata_disk->getPath());
|
||||
|
||||
remote_fs_object_path = remote_fs_object_path.substr(remote_fs_root_path.size());
|
||||
}
|
||||
assertChar('\n', *buf);
|
||||
remote_fs_objects[i].relative_path = remote_fs_object_path;
|
||||
remote_fs_objects[i].bytes_size = remote_fs_object_size;
|
||||
}
|
||||
|
||||
readIntText(ref_count, *buf);
|
||||
assertChar('\n', *buf);
|
||||
|
||||
if (version >= VERSION_READ_ONLY_FLAG)
|
||||
{
|
||||
readBoolText(read_only, *buf);
|
||||
assertChar('\n', *buf);
|
||||
}
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
|
||||
if (e.code() == ErrorCodes::UNKNOWN_FORMAT)
|
||||
throw;
|
||||
|
||||
if (e.code() == ErrorCodes::MEMORY_LIMIT_EXCEEDED)
|
||||
throw;
|
||||
|
||||
throw Exception("Failed to read metadata file: " + metadata_file_path, ErrorCodes::UNKNOWN_FORMAT);
|
||||
}
|
||||
}
|
||||
|
||||
/// Load metadata by path or create empty if `create` flag is set.
|
||||
DiskObjectStorageMetadata::DiskObjectStorageMetadata(
|
||||
const String & remote_fs_root_path_,
|
||||
DiskPtr metadata_disk_,
|
||||
const String & metadata_file_path_)
|
||||
: remote_fs_root_path(remote_fs_root_path_)
|
||||
, metadata_file_path(metadata_file_path_)
|
||||
, metadata_disk(metadata_disk_)
|
||||
, total_size(0), ref_count(0)
|
||||
{
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadata::addObject(const String & path, size_t size)
|
||||
{
|
||||
total_size += size;
|
||||
remote_fs_objects.emplace_back(path, size);
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorageMetadata::saveToBuffer(WriteBuffer & buf, bool sync)
|
||||
{
|
||||
writeIntText(VERSION_RELATIVE_PATHS, buf);
|
||||
writeChar('\n', buf);
|
||||
|
||||
writeIntText(remote_fs_objects.size(), buf);
|
||||
writeChar('\t', buf);
|
||||
writeIntText(total_size, buf);
|
||||
writeChar('\n', buf);
|
||||
|
||||
for (const auto & [remote_fs_object_path, remote_fs_object_size] : remote_fs_objects)
|
||||
{
|
||||
writeIntText(remote_fs_object_size, buf);
|
||||
writeChar('\t', buf);
|
||||
writeEscapedString(remote_fs_object_path, buf);
|
||||
writeChar('\n', buf);
|
||||
}
|
||||
|
||||
writeIntText(ref_count, buf);
|
||||
writeChar('\n', buf);
|
||||
|
||||
writeBoolText(read_only, buf);
|
||||
writeChar('\n', buf);
|
||||
|
||||
buf.finalize();
|
||||
if (sync)
|
||||
buf.sync();
|
||||
|
||||
}
|
||||
|
||||
/// Fsync metadata file if 'sync' flag is set.
|
||||
void DiskObjectStorageMetadata::save(bool sync)
|
||||
{
|
||||
auto buf = metadata_disk->writeFile(metadata_file_path, 1024);
|
||||
saveToBuffer(*buf, sync);
|
||||
}
|
||||
|
||||
std::string DiskObjectStorageMetadata::serializeToString()
|
||||
{
|
||||
WriteBufferFromOwnString write_buf;
|
||||
saveToBuffer(write_buf, false);
|
||||
return write_buf.str();
|
||||
}
|
||||
|
||||
|
||||
}
|
68
src/Disks/ObjectStorages/DiskObjectStorageMetadata.h
Normal file
68
src/Disks/ObjectStorages/DiskObjectStorageMetadata.h
Normal file
@ -0,0 +1,68 @@
|
||||
#pragma once
|
||||
|
||||
#include <Disks/IDisk.h>
|
||||
#include <Core/Types.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Metadata for DiskObjectStorage, stored on local disk
|
||||
struct DiskObjectStorageMetadata
|
||||
{
|
||||
using Updater = std::function<bool(DiskObjectStorageMetadata & metadata)>;
|
||||
/// Metadata file version.
|
||||
static constexpr UInt32 VERSION_ABSOLUTE_PATHS = 1;
|
||||
static constexpr UInt32 VERSION_RELATIVE_PATHS = 2;
|
||||
static constexpr UInt32 VERSION_READ_ONLY_FLAG = 3;
|
||||
|
||||
/// Remote FS objects paths and their sizes.
|
||||
std::vector<BlobPathWithSize> remote_fs_objects;
|
||||
|
||||
/// URI
|
||||
const String & remote_fs_root_path;
|
||||
|
||||
/// Relative path to metadata file on local FS.
|
||||
const String metadata_file_path;
|
||||
|
||||
DiskPtr metadata_disk;
|
||||
|
||||
/// Total size of all remote FS (S3, HDFS) objects.
|
||||
size_t total_size = 0;
|
||||
|
||||
/// Number of references (hardlinks) to this metadata file.
|
||||
///
|
||||
/// FIXME: Why we are tracking it explicetly, without
|
||||
/// info from filesystem????
|
||||
UInt32 ref_count = 0;
|
||||
|
||||
/// Flag indicates that file is read only.
|
||||
bool read_only = false;
|
||||
|
||||
DiskObjectStorageMetadata(
|
||||
const String & remote_fs_root_path_,
|
||||
DiskPtr metadata_disk_,
|
||||
const String & metadata_file_path_);
|
||||
|
||||
void addObject(const String & path, size_t size);
|
||||
|
||||
static DiskObjectStorageMetadata readMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_);
|
||||
static DiskObjectStorageMetadata readUpdateAndStoreMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, Updater updater);
|
||||
static DiskObjectStorageMetadata readUpdateStoreMetadataAndRemove(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, Updater updater);
|
||||
|
||||
static DiskObjectStorageMetadata createAndStoreMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync);
|
||||
static DiskObjectStorageMetadata createUpdateAndStoreMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, Updater updater);
|
||||
static DiskObjectStorageMetadata createAndStoreMetadataIfNotExists(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, bool overwrite);
|
||||
|
||||
/// Serialize metadata to string (very same with saveToBuffer)
|
||||
std::string serializeToString();
|
||||
|
||||
private:
|
||||
/// Fsync metadata file if 'sync' flag is set.
|
||||
void save(bool sync = false);
|
||||
void saveToBuffer(WriteBuffer & buffer, bool sync);
|
||||
void load();
|
||||
};
|
||||
|
||||
using DiskObjectStorageMetadataUpdater = std::function<bool(DiskObjectStorageMetadata & metadata)>;
|
||||
|
||||
}
|
571
src/Disks/ObjectStorages/DiskObjectStorageMetadataHelper.cpp
Normal file
571
src/Disks/ObjectStorages/DiskObjectStorageMetadataHelper.cpp
Normal file
@ -0,0 +1,571 @@
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageMetadataHelper.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorage.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <Common/checkStackSize.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNKNOWN_FORMAT;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
static String revisionToString(UInt64 revision)
|
||||
{
|
||||
return std::bitset<64>(revision).to_string();
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::createFileOperationObject(const String & operation_name, UInt64 revision, const ObjectAttributes & metadata) const
|
||||
{
|
||||
const String path = disk->remote_fs_root_path + "operations/r" + revisionToString(revision) + operation_log_suffix + "-" + operation_name;
|
||||
auto buf = disk->object_storage->writeObject(path, WriteMode::Rewrite, metadata);
|
||||
buf->write('0');
|
||||
buf->finalize();
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::findLastRevision()
|
||||
{
|
||||
/// Construct revision number from high to low bits.
|
||||
String revision;
|
||||
revision.reserve(64);
|
||||
for (int bit = 0; bit < 64; ++bit)
|
||||
{
|
||||
auto revision_prefix = revision + "1";
|
||||
|
||||
LOG_TRACE(disk->log, "Check object exists with revision prefix {}", revision_prefix);
|
||||
|
||||
/// Check file or operation with such revision prefix exists.
|
||||
if (disk->object_storage->exists(disk->remote_fs_root_path + "r" + revision_prefix)
|
||||
|| disk->object_storage->exists(disk->remote_fs_root_path + "operations/r" + revision_prefix))
|
||||
revision += "1";
|
||||
else
|
||||
revision += "0";
|
||||
}
|
||||
revision_counter = static_cast<UInt64>(std::bitset<64>(revision).to_ullong());
|
||||
LOG_INFO(disk->log, "Found last revision number {} for disk {}", revision_counter, disk->name);
|
||||
}
|
||||
|
||||
int DiskObjectStorageMetadataHelper::readSchemaVersion(IObjectStorage * object_storage, const String & source_path)
|
||||
{
|
||||
const std::string path = source_path + SCHEMA_VERSION_OBJECT;
|
||||
int version = 0;
|
||||
if (!object_storage->exists(path))
|
||||
return version;
|
||||
|
||||
auto buf = object_storage->readObject(path);
|
||||
readIntText(version, *buf);
|
||||
|
||||
return version;
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::saveSchemaVersion(const int & version) const
|
||||
{
|
||||
auto path = disk->remote_fs_root_path + SCHEMA_VERSION_OBJECT;
|
||||
|
||||
auto buf = disk->object_storage->writeObject(path, WriteMode::Rewrite);
|
||||
writeIntText(version, *buf);
|
||||
buf->finalize();
|
||||
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::updateObjectMetadata(const String & key, const ObjectAttributes & metadata) const
|
||||
{
|
||||
disk->object_storage->copyObject(key, key, metadata);
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::migrateFileToRestorableSchema(const String & path) const
|
||||
{
|
||||
LOG_TRACE(disk->log, "Migrate file {} to restorable schema", disk->metadata_disk->getPath() + path);
|
||||
|
||||
auto meta = disk->readMetadata(path);
|
||||
|
||||
for (const auto & [key, _] : meta.remote_fs_objects)
|
||||
{
|
||||
ObjectAttributes metadata {
|
||||
{"path", path}
|
||||
};
|
||||
updateObjectMetadata(disk->remote_fs_root_path + key, metadata);
|
||||
}
|
||||
}
|
||||
void DiskObjectStorageMetadataHelper::migrateToRestorableSchemaRecursive(const String & path, Futures & results)
|
||||
{
|
||||
checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks.
|
||||
|
||||
LOG_TRACE(disk->log, "Migrate directory {} to restorable schema", disk->metadata_disk->getPath() + path);
|
||||
|
||||
bool dir_contains_only_files = true;
|
||||
for (auto it = disk->iterateDirectory(path); it->isValid(); it->next())
|
||||
{
|
||||
if (disk->isDirectory(it->path()))
|
||||
{
|
||||
dir_contains_only_files = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
/// The whole directory can be migrated asynchronously.
|
||||
if (dir_contains_only_files)
|
||||
{
|
||||
auto result = disk->getExecutor().execute([this, path]
|
||||
{
|
||||
for (auto it = disk->iterateDirectory(path); it->isValid(); it->next())
|
||||
migrateFileToRestorableSchema(it->path());
|
||||
});
|
||||
|
||||
results.push_back(std::move(result));
|
||||
}
|
||||
else
|
||||
{
|
||||
for (auto it = disk->iterateDirectory(path); it->isValid(); it->next())
|
||||
if (!disk->isDirectory(it->path()))
|
||||
{
|
||||
auto source_path = it->path();
|
||||
auto result = disk->getExecutor().execute([this, source_path]
|
||||
{
|
||||
migrateFileToRestorableSchema(source_path);
|
||||
});
|
||||
|
||||
results.push_back(std::move(result));
|
||||
}
|
||||
else
|
||||
migrateToRestorableSchemaRecursive(it->path(), results);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::migrateToRestorableSchema()
|
||||
{
|
||||
try
|
||||
{
|
||||
LOG_INFO(disk->log, "Start migration to restorable schema for disk {}", disk->name);
|
||||
|
||||
Futures results;
|
||||
|
||||
for (const auto & root : data_roots)
|
||||
if (disk->exists(root))
|
||||
migrateToRestorableSchemaRecursive(root + '/', results);
|
||||
|
||||
for (auto & result : results)
|
||||
result.wait();
|
||||
for (auto & result : results)
|
||||
result.get();
|
||||
|
||||
saveSchemaVersion(RESTORABLE_SCHEMA_VERSION);
|
||||
}
|
||||
catch (const Exception &)
|
||||
{
|
||||
tryLogCurrentException(disk->log, fmt::format("Failed to migrate to restorable schema for disk {}", disk->name));
|
||||
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::restore(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context)
|
||||
{
|
||||
LOG_INFO(disk->log, "Restore operation for disk {} called", disk->name);
|
||||
|
||||
if (!disk->exists(RESTORE_FILE_NAME))
|
||||
{
|
||||
LOG_INFO(disk->log, "No restore file '{}' exists, finishing restore", RESTORE_FILE_NAME);
|
||||
return;
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
RestoreInformation information;
|
||||
information.source_path = disk->remote_fs_root_path;
|
||||
information.source_namespace = disk->object_storage->getObjectsNamespace();
|
||||
|
||||
readRestoreInformation(information);
|
||||
if (information.revision == 0)
|
||||
information.revision = LATEST_REVISION;
|
||||
if (!information.source_path.ends_with('/'))
|
||||
information.source_path += '/';
|
||||
|
||||
IObjectStorage * source_object_storage = disk->object_storage.get();
|
||||
if (information.source_namespace == disk->object_storage->getObjectsNamespace())
|
||||
{
|
||||
/// In this case we need to additionally cleanup S3 from objects with later revision.
|
||||
/// Will be simply just restore to different path.
|
||||
if (information.source_path == disk->remote_fs_root_path && information.revision != LATEST_REVISION)
|
||||
throw Exception("Restoring to the same bucket and path is allowed if revision is latest (0)", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
/// This case complicates S3 cleanup in case of unsuccessful restore.
|
||||
if (information.source_path != disk->remote_fs_root_path && disk->remote_fs_root_path.starts_with(information.source_path))
|
||||
throw Exception("Restoring to the same bucket is allowed only if source path is not a sub-path of configured path in S3 disk", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
else
|
||||
{
|
||||
object_storage_from_another_namespace = disk->object_storage->cloneObjectStorage(information.source_namespace, config, config_prefix, context);
|
||||
source_object_storage = object_storage_from_another_namespace.get();
|
||||
}
|
||||
|
||||
LOG_INFO(disk->log, "Starting to restore disk {}. Revision: {}, Source path: {}",
|
||||
disk->name, information.revision, information.source_path);
|
||||
|
||||
if (readSchemaVersion(source_object_storage, information.source_path) < RESTORABLE_SCHEMA_VERSION)
|
||||
throw Exception("Source bucket doesn't have restorable schema.", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
LOG_INFO(disk->log, "Removing old metadata...");
|
||||
|
||||
bool cleanup_s3 = information.source_path != disk->remote_fs_root_path;
|
||||
for (const auto & root : data_roots)
|
||||
if (disk->exists(root))
|
||||
disk->removeSharedRecursive(root + '/', !cleanup_s3, {});
|
||||
|
||||
LOG_INFO(disk->log, "Old metadata removed, restoring new one");
|
||||
restoreFiles(source_object_storage, information);
|
||||
restoreFileOperations(source_object_storage, information);
|
||||
|
||||
disk->metadata_disk->removeFile(RESTORE_FILE_NAME);
|
||||
|
||||
saveSchemaVersion(RESTORABLE_SCHEMA_VERSION);
|
||||
|
||||
LOG_INFO(disk->log, "Restore disk {} finished", disk->name);
|
||||
}
|
||||
catch (const Exception &)
|
||||
{
|
||||
tryLogCurrentException(disk->log, fmt::format("Failed to restore disk {}", disk->name));
|
||||
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::readRestoreInformation(RestoreInformation & restore_information) /// NOLINT
|
||||
{
|
||||
auto buffer = disk->metadata_disk->readFile(RESTORE_FILE_NAME, ReadSettings{}, 512);
|
||||
buffer->next();
|
||||
|
||||
try
|
||||
{
|
||||
std::map<String, String> properties;
|
||||
|
||||
while (buffer->hasPendingData())
|
||||
{
|
||||
String property;
|
||||
readText(property, *buffer);
|
||||
assertChar('\n', *buffer);
|
||||
|
||||
auto pos = property.find('=');
|
||||
if (pos == std::string::npos || pos == 0 || pos == property.length())
|
||||
throw Exception(fmt::format("Invalid property {} in restore file", property), ErrorCodes::UNKNOWN_FORMAT);
|
||||
|
||||
auto key = property.substr(0, pos);
|
||||
auto value = property.substr(pos + 1);
|
||||
|
||||
auto it = properties.find(key);
|
||||
if (it != properties.end())
|
||||
throw Exception(fmt::format("Property key duplication {} in restore file", key), ErrorCodes::UNKNOWN_FORMAT);
|
||||
|
||||
properties[key] = value;
|
||||
}
|
||||
|
||||
for (const auto & [key, value] : properties)
|
||||
{
|
||||
ReadBufferFromString value_buffer(value);
|
||||
|
||||
if (key == "revision")
|
||||
readIntText(restore_information.revision, value_buffer);
|
||||
else if (key == "source_bucket" || key == "source_namespace")
|
||||
readText(restore_information.source_namespace, value_buffer);
|
||||
else if (key == "source_path")
|
||||
readText(restore_information.source_path, value_buffer);
|
||||
else if (key == "detached")
|
||||
readBoolTextWord(restore_information.detached, value_buffer);
|
||||
else
|
||||
throw Exception(fmt::format("Unknown key {} in restore file", key), ErrorCodes::UNKNOWN_FORMAT);
|
||||
}
|
||||
}
|
||||
catch (const Exception &)
|
||||
{
|
||||
tryLogCurrentException(disk->log, "Failed to read restore information");
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
static String shrinkKey(const String & path, const String & key)
|
||||
{
|
||||
if (!key.starts_with(path))
|
||||
throw Exception("The key " + key + " prefix mismatch with given " + path, ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return key.substr(path.length());
|
||||
}
|
||||
|
||||
static std::tuple<UInt64, String> extractRevisionAndOperationFromKey(const String & key)
|
||||
{
|
||||
String revision_str;
|
||||
String suffix;
|
||||
String operation;
|
||||
/// Key has format: ../../r{revision}(-{hostname})-{operation}
|
||||
static const re2::RE2 key_regexp{R"(.*/r(\d+)(-[\w\d\-\.]+)?-(\w+)$)"};
|
||||
|
||||
re2::RE2::FullMatch(key, key_regexp, &revision_str, &suffix, &operation);
|
||||
|
||||
return {(revision_str.empty() ? 0 : static_cast<UInt64>(std::bitset<64>(revision_str).to_ullong())), operation};
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::moveRecursiveOrRemove(const String & from_path, const String & to_path, bool send_metadata)
|
||||
{
|
||||
if (disk->exists(to_path))
|
||||
{
|
||||
if (send_metadata)
|
||||
{
|
||||
auto revision = ++revision_counter;
|
||||
const ObjectAttributes object_metadata {
|
||||
{"from_path", from_path},
|
||||
{"to_path", to_path}
|
||||
};
|
||||
createFileOperationObject("rename", revision, object_metadata);
|
||||
}
|
||||
if (disk->isDirectory(from_path))
|
||||
{
|
||||
for (auto it = disk->iterateDirectory(from_path); it->isValid(); it->next())
|
||||
moveRecursiveOrRemove(it->path(), fs::path(to_path) / it->name(), false);
|
||||
}
|
||||
else
|
||||
{
|
||||
disk->removeFile(from_path);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
disk->moveFile(from_path, to_path, send_metadata);
|
||||
}
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::restoreFiles(IObjectStorage * source_object_storage, const RestoreInformation & restore_information)
|
||||
{
|
||||
LOG_INFO(disk->log, "Starting restore files for disk {}", disk->name);
|
||||
|
||||
std::vector<std::future<void>> results;
|
||||
auto restore_files = [this, &source_object_storage, &restore_information, &results](const BlobsPathToSize & keys)
|
||||
{
|
||||
std::vector<String> keys_names;
|
||||
for (const auto & [key, size] : keys)
|
||||
{
|
||||
|
||||
LOG_INFO(disk->log, "Calling restore for key for disk {}", key);
|
||||
|
||||
/// Skip file operations objects. They will be processed separately.
|
||||
if (key.find("/operations/") != String::npos)
|
||||
continue;
|
||||
|
||||
const auto [revision, _] = extractRevisionAndOperationFromKey(key);
|
||||
/// Filter early if it's possible to get revision from key.
|
||||
if (revision > restore_information.revision)
|
||||
continue;
|
||||
|
||||
keys_names.push_back(key);
|
||||
}
|
||||
|
||||
if (!keys_names.empty())
|
||||
{
|
||||
auto result = disk->getExecutor().execute([this, &source_object_storage, &restore_information, keys_names]()
|
||||
{
|
||||
processRestoreFiles(source_object_storage, restore_information.source_path, keys_names);
|
||||
});
|
||||
|
||||
results.push_back(std::move(result));
|
||||
}
|
||||
|
||||
return true;
|
||||
};
|
||||
|
||||
BlobsPathToSize children;
|
||||
source_object_storage->listPrefix(restore_information.source_path, children);
|
||||
|
||||
restore_files(children);
|
||||
|
||||
for (auto & result : results)
|
||||
result.wait();
|
||||
for (auto & result : results)
|
||||
result.get();
|
||||
|
||||
LOG_INFO(disk->log, "Files are restored for disk {}", disk->name);
|
||||
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::processRestoreFiles(IObjectStorage * source_object_storage, const String & source_path, const std::vector<String> & keys) const
|
||||
{
|
||||
for (const auto & key : keys)
|
||||
{
|
||||
auto meta = source_object_storage->getObjectMetadata(key);
|
||||
auto object_attributes = meta.attributes;
|
||||
|
||||
String path;
|
||||
if (object_attributes.has_value())
|
||||
{
|
||||
/// Restore file if object has 'path' in metadata.
|
||||
auto path_entry = object_attributes->find("path");
|
||||
if (path_entry == object_attributes->end())
|
||||
{
|
||||
/// Such keys can remain after migration, we can skip them.
|
||||
LOG_WARNING(disk->log, "Skip key {} because it doesn't have 'path' in metadata", key);
|
||||
continue;
|
||||
}
|
||||
|
||||
path = path_entry->second;
|
||||
}
|
||||
else
|
||||
continue;
|
||||
|
||||
disk->createDirectories(directoryPath(path));
|
||||
auto relative_key = shrinkKey(source_path, key);
|
||||
|
||||
/// Copy object if we restore to different bucket / path.
|
||||
if (source_object_storage->getObjectsNamespace() != disk->object_storage->getObjectsNamespace() || disk->remote_fs_root_path != source_path)
|
||||
source_object_storage->copyObjectToAnotherObjectStorage(key, disk->remote_fs_root_path + relative_key, *disk->object_storage);
|
||||
|
||||
auto updater = [relative_key, meta] (DiskObjectStorage::Metadata & metadata)
|
||||
{
|
||||
metadata.addObject(relative_key, meta.size_bytes);
|
||||
return true;
|
||||
};
|
||||
|
||||
disk->createUpdateAndStoreMetadata(path, false, updater);
|
||||
|
||||
LOG_TRACE(disk->log, "Restored file {}", path);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void DiskObjectStorage::onFreeze(const String & path)
|
||||
{
|
||||
createDirectories(path);
|
||||
auto revision_file_buf = metadata_disk->writeFile(path + "revision.txt", 32);
|
||||
writeIntText(metadata_helper->revision_counter.load(), *revision_file_buf);
|
||||
revision_file_buf->finalize();
|
||||
}
|
||||
|
||||
static String pathToDetached(const String & source_path)
|
||||
{
|
||||
if (source_path.ends_with('/'))
|
||||
return fs::path(source_path).parent_path().parent_path() / "detached/";
|
||||
return fs::path(source_path).parent_path() / "detached/";
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::restoreFileOperations(IObjectStorage * source_object_storage, const RestoreInformation & restore_information)
|
||||
{
|
||||
/// Enable recording file operations if we restore to different bucket / path.
|
||||
bool send_metadata = source_object_storage->getObjectsNamespace() != disk->object_storage->getObjectsNamespace() || disk->remote_fs_root_path != restore_information.source_path;
|
||||
|
||||
std::set<String> renames;
|
||||
auto restore_file_operations = [this, &source_object_storage, &restore_information, &renames, &send_metadata](const BlobsPathToSize & keys)
|
||||
{
|
||||
const String rename = "rename";
|
||||
const String hardlink = "hardlink";
|
||||
|
||||
for (const auto & [key, _]: keys)
|
||||
{
|
||||
const auto [revision, operation] = extractRevisionAndOperationFromKey(key);
|
||||
if (revision == UNKNOWN_REVISION)
|
||||
{
|
||||
LOG_WARNING(disk->log, "Skip key {} with unknown revision", key);
|
||||
continue;
|
||||
}
|
||||
|
||||
/// S3 ensures that keys will be listed in ascending UTF-8 bytes order (revision order).
|
||||
/// We can stop processing if revision of the object is already more than required.
|
||||
if (revision > restore_information.revision)
|
||||
return false;
|
||||
|
||||
/// Keep original revision if restore to different bucket / path.
|
||||
if (send_metadata)
|
||||
revision_counter = revision - 1;
|
||||
|
||||
auto object_attributes = *(source_object_storage->getObjectMetadata(key).attributes);
|
||||
if (operation == rename)
|
||||
{
|
||||
auto from_path = object_attributes["from_path"];
|
||||
auto to_path = object_attributes["to_path"];
|
||||
if (disk->exists(from_path))
|
||||
{
|
||||
moveRecursiveOrRemove(from_path, to_path, send_metadata);
|
||||
|
||||
LOG_TRACE(disk->log, "Revision {}. Restored rename {} -> {}", revision, from_path, to_path);
|
||||
|
||||
if (restore_information.detached && disk->isDirectory(to_path))
|
||||
{
|
||||
/// Sometimes directory paths are passed without trailing '/'. We should keep them in one consistent way.
|
||||
if (!from_path.ends_with('/'))
|
||||
from_path += '/';
|
||||
if (!to_path.ends_with('/'))
|
||||
to_path += '/';
|
||||
|
||||
/// Always keep latest actual directory path to avoid 'detaching' not existing paths.
|
||||
auto it = renames.find(from_path);
|
||||
if (it != renames.end())
|
||||
renames.erase(it);
|
||||
|
||||
renames.insert(to_path);
|
||||
}
|
||||
}
|
||||
}
|
||||
else if (operation == hardlink)
|
||||
{
|
||||
auto src_path = object_attributes["src_path"];
|
||||
auto dst_path = object_attributes["dst_path"];
|
||||
if (disk->exists(src_path))
|
||||
{
|
||||
disk->createDirectories(directoryPath(dst_path));
|
||||
disk->createHardLink(src_path, dst_path, send_metadata);
|
||||
LOG_TRACE(disk->log, "Revision {}. Restored hardlink {} -> {}", revision, src_path, dst_path);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
};
|
||||
|
||||
BlobsPathToSize children;
|
||||
source_object_storage->listPrefix(restore_information.source_path + "operations/", children);
|
||||
restore_file_operations(children);
|
||||
|
||||
if (restore_information.detached)
|
||||
{
|
||||
Strings not_finished_prefixes{"tmp_", "delete_tmp_", "attaching_", "deleting_"};
|
||||
|
||||
for (const auto & path : renames)
|
||||
{
|
||||
/// Skip already detached parts.
|
||||
if (path.find("/detached/") != std::string::npos)
|
||||
continue;
|
||||
|
||||
/// Skip not finished parts. They shouldn't be in 'detached' directory, because CH wouldn't be able to finish processing them.
|
||||
fs::path directory_path(path);
|
||||
auto directory_name = directory_path.parent_path().filename().string();
|
||||
|
||||
auto predicate = [&directory_name](String & prefix) { return directory_name.starts_with(prefix); };
|
||||
if (std::any_of(not_finished_prefixes.begin(), not_finished_prefixes.end(), predicate))
|
||||
continue;
|
||||
|
||||
auto detached_path = pathToDetached(path);
|
||||
|
||||
LOG_TRACE(disk->log, "Move directory to 'detached' {} -> {}", path, detached_path);
|
||||
|
||||
fs::path from_path = fs::path(path);
|
||||
fs::path to_path = fs::path(detached_path);
|
||||
if (path.ends_with('/'))
|
||||
to_path /= from_path.parent_path().filename();
|
||||
else
|
||||
to_path /= from_path.filename();
|
||||
|
||||
/// to_path may exist and non-empty in case for example abrupt restart, so remove it before rename
|
||||
if (disk->metadata_disk->exists(to_path))
|
||||
disk->metadata_disk->removeRecursive(to_path);
|
||||
|
||||
disk->createDirectories(directoryPath(to_path));
|
||||
disk->metadata_disk->moveDirectory(from_path, to_path);
|
||||
}
|
||||
}
|
||||
|
||||
LOG_INFO(disk->log, "File operations restored for disk {}", disk->name);
|
||||
}
|
||||
|
||||
}
|
100
src/Disks/ObjectStorages/DiskObjectStorageMetadataHelper.h
Normal file
100
src/Disks/ObjectStorages/DiskObjectStorageMetadataHelper.h
Normal file
@ -0,0 +1,100 @@
|
||||
#pragma once
|
||||
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class DiskObjectStorage;
|
||||
|
||||
/// Class implements storage of ObjectStorage metadata inside object storage itself,
|
||||
/// so it's possible to recover from this remote information in case of local disk loss.
|
||||
///
|
||||
/// This mechanism can be enabled with `<send_metadata>true</send_metadata>` option inside
|
||||
/// disk configuration. Implemented only for S3 and Azure Blob storage. Other object storages
|
||||
/// don't support metadata for blobs.
|
||||
///
|
||||
/// FIXME: this class is very intrusive and use a lot of DiskObjectStorage internals.
|
||||
/// FIXME: it's very complex and unreliable, need to implement something better.
|
||||
class DiskObjectStorageMetadataHelper
|
||||
{
|
||||
public:
|
||||
static constexpr UInt64 LATEST_REVISION = std::numeric_limits<UInt64>::max();
|
||||
static constexpr UInt64 UNKNOWN_REVISION = 0;
|
||||
|
||||
DiskObjectStorageMetadataHelper(DiskObjectStorage * disk_, ReadSettings read_settings_)
|
||||
: disk(disk_)
|
||||
, read_settings(std::move(read_settings_))
|
||||
, operation_log_suffix("-" + getFQDNOrHostName())
|
||||
{
|
||||
}
|
||||
|
||||
/// Most important method, called on DiskObjectStorage startup
|
||||
void restore(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context);
|
||||
|
||||
void syncRevision(UInt64 revision)
|
||||
{
|
||||
UInt64 local_revision = revision_counter.load();
|
||||
while ((revision > local_revision) && revision_counter.compare_exchange_weak(local_revision, revision));
|
||||
}
|
||||
|
||||
UInt64 getRevision() const
|
||||
{
|
||||
return revision_counter.load();
|
||||
}
|
||||
|
||||
static int readSchemaVersion(IObjectStorage * object_storage, const String & source_path);
|
||||
|
||||
void migrateToRestorableSchema();
|
||||
|
||||
void findLastRevision();
|
||||
|
||||
void createFileOperationObject(const String & operation_name, UInt64 revision, const ObjectAttributes & metadata) const;
|
||||
|
||||
/// Version with possibility to backup-restore metadata.
|
||||
static constexpr int RESTORABLE_SCHEMA_VERSION = 1;
|
||||
|
||||
std::atomic<UInt64> revision_counter = 0;
|
||||
private:
|
||||
struct RestoreInformation
|
||||
{
|
||||
UInt64 revision = LATEST_REVISION;
|
||||
String source_namespace;
|
||||
String source_path;
|
||||
bool detached = false;
|
||||
};
|
||||
|
||||
using Futures = std::vector<std::future<void>>;
|
||||
|
||||
/// 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);
|
||||
|
||||
void saveSchemaVersion(const int & version) const;
|
||||
void updateObjectMetadata(const String & key, const ObjectAttributes & metadata) const;
|
||||
void migrateFileToRestorableSchema(const String & path) const;
|
||||
void migrateToRestorableSchemaRecursive(const String & path, Futures & results);
|
||||
|
||||
void readRestoreInformation(RestoreInformation & restore_information);
|
||||
void restoreFiles(IObjectStorage * source_object_storage, const RestoreInformation & restore_information);
|
||||
void processRestoreFiles(IObjectStorage * source_object_storage, const String & source_path, const std::vector<String> & keys) const;
|
||||
void restoreFileOperations(IObjectStorage * source_object_storage, const RestoreInformation & restore_information);
|
||||
|
||||
inline static const String RESTORE_FILE_NAME = "restore";
|
||||
|
||||
/// Object contains information about schema version.
|
||||
inline static const String SCHEMA_VERSION_OBJECT = ".SCHEMA_VERSION";
|
||||
/// Directories with data.
|
||||
const std::vector<String> data_roots {"data", "store"};
|
||||
|
||||
DiskObjectStorage * disk;
|
||||
|
||||
ObjectStoragePtr object_storage_from_another_namespace;
|
||||
|
||||
ReadSettings read_settings;
|
||||
|
||||
String operation_log_suffix;
|
||||
};
|
||||
|
||||
}
|
156
src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp
Normal file
156
src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp
Normal file
@ -0,0 +1,156 @@
|
||||
#include <Disks/ObjectStorages/HDFS/HDFSObjectStorage.h>
|
||||
|
||||
#include <IO/SeekAvoidingReadBuffer.h>
|
||||
#include <IO/copyData.h>
|
||||
|
||||
#include <Storages/HDFS/WriteBufferFromHDFS.h>
|
||||
#include <Storages/HDFS/HDFSCommon.h>
|
||||
|
||||
#include <Storages/HDFS/ReadBufferFromHDFS.h>
|
||||
#include <Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/ReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/WriteIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
|
||||
|
||||
|
||||
#if USE_HDFS
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
extern const int HDFS_ERROR;
|
||||
}
|
||||
|
||||
void HDFSObjectStorage::shutdown()
|
||||
{
|
||||
}
|
||||
|
||||
void HDFSObjectStorage::startup()
|
||||
{
|
||||
}
|
||||
|
||||
bool HDFSObjectStorage::exists(const std::string & hdfs_uri) const
|
||||
{
|
||||
const size_t begin_of_path = hdfs_uri.find('/', hdfs_uri.find("//") + 2);
|
||||
const String remote_fs_object_path = hdfs_uri.substr(begin_of_path);
|
||||
return (0 == hdfsExists(hdfs_fs.get(), remote_fs_object_path.c_str()));
|
||||
}
|
||||
|
||||
std::unique_ptr<SeekableReadBuffer> HDFSObjectStorage::readObject( /// NOLINT
|
||||
const std::string & path,
|
||||
const ReadSettings & read_settings,
|
||||
std::optional<size_t>,
|
||||
std::optional<size_t>) const
|
||||
{
|
||||
return std::make_unique<ReadBufferFromHDFS>(path, path, config, read_settings.remote_fs_buffer_size);
|
||||
}
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> HDFSObjectStorage::readObjects( /// NOLINT
|
||||
const std::string & common_path_prefix,
|
||||
const BlobsPathToSize & blobs_to_read,
|
||||
const ReadSettings & read_settings,
|
||||
std::optional<size_t>,
|
||||
std::optional<size_t>) const
|
||||
{
|
||||
auto hdfs_impl = std::make_unique<ReadBufferFromHDFSGather>(config, common_path_prefix, common_path_prefix, blobs_to_read, read_settings);
|
||||
auto buf = std::make_unique<ReadIndirectBufferFromRemoteFS>(std::move(hdfs_impl));
|
||||
return std::make_unique<SeekAvoidingReadBuffer>(std::move(buf), settings->min_bytes_for_seek);
|
||||
}
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> HDFSObjectStorage::writeObject( /// NOLINT
|
||||
const std::string & path,
|
||||
WriteMode mode,
|
||||
std::optional<ObjectAttributes> attributes,
|
||||
FinalizeCallback && finalize_callback,
|
||||
size_t buf_size,
|
||||
const WriteSettings &)
|
||||
{
|
||||
if (attributes.has_value())
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "HDFS API doesn't support custom attributes/metadata for stored objects");
|
||||
|
||||
/// Single O_WRONLY in libhdfs adds O_TRUNC
|
||||
auto hdfs_buffer = std::make_unique<WriteBufferFromHDFS>(
|
||||
path, config, settings->replication, buf_size,
|
||||
mode == WriteMode::Rewrite ? O_WRONLY : O_WRONLY | O_APPEND);
|
||||
|
||||
return std::make_unique<WriteIndirectBufferFromRemoteFS>(std::move(hdfs_buffer), std::move(finalize_callback), path);
|
||||
}
|
||||
|
||||
|
||||
void HDFSObjectStorage::listPrefix(const std::string & path, BlobsPathToSize & children) const
|
||||
{
|
||||
const size_t begin_of_path = path.find('/', path.find("//") + 2);
|
||||
int32_t num_entries;
|
||||
auto * files_list = hdfsListDirectory(hdfs_fs.get(), path.substr(begin_of_path).c_str(), &num_entries);
|
||||
if (num_entries == -1)
|
||||
throw Exception(ErrorCodes::HDFS_ERROR, "HDFSDelete failed with path: " + path);
|
||||
|
||||
for (int32_t i = 0; i < num_entries; ++i)
|
||||
children.emplace_back(files_list[i].mName, files_list[i].mSize);
|
||||
}
|
||||
|
||||
/// Remove file. Throws exception if file doesn't exists or it's a directory.
|
||||
void HDFSObjectStorage::removeObject(const std::string & path)
|
||||
{
|
||||
const size_t begin_of_path = path.find('/', path.find("//") + 2);
|
||||
|
||||
/// Add path from root to file name
|
||||
int res = hdfsDelete(hdfs_fs.get(), path.substr(begin_of_path).c_str(), 0);
|
||||
if (res == -1)
|
||||
throw Exception(ErrorCodes::HDFS_ERROR, "HDFSDelete failed with path: " + path);
|
||||
|
||||
}
|
||||
|
||||
void HDFSObjectStorage::removeObjects(const std::vector<std::string> & paths)
|
||||
{
|
||||
for (const auto & hdfs_path : paths)
|
||||
removeObject(hdfs_path);
|
||||
}
|
||||
|
||||
void HDFSObjectStorage::removeObjectIfExists(const std::string & path)
|
||||
{
|
||||
if (exists(path))
|
||||
removeObject(path);
|
||||
}
|
||||
|
||||
void HDFSObjectStorage::removeObjectsIfExist(const std::vector<std::string> & paths)
|
||||
{
|
||||
for (const auto & hdfs_path : paths)
|
||||
removeObjectIfExists(hdfs_path);
|
||||
}
|
||||
|
||||
ObjectMetadata HDFSObjectStorage::getObjectMetadata(const std::string &) const
|
||||
{
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "HDFS API doesn't support custom attributes/metadata for stored objects");
|
||||
}
|
||||
|
||||
void HDFSObjectStorage::copyObject( /// NOLINT
|
||||
const std::string & object_from,
|
||||
const std::string & object_to,
|
||||
std::optional<ObjectAttributes> object_to_attributes)
|
||||
{
|
||||
if (object_to_attributes.has_value())
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "HDFS API doesn't support custom attributes/metadata for stored objects");
|
||||
|
||||
auto in = readObject(object_from);
|
||||
auto out = writeObject(object_to, WriteMode::Rewrite);
|
||||
copyData(*in, *out);
|
||||
out->finalize();
|
||||
}
|
||||
|
||||
|
||||
void HDFSObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration &, const std::string &, ContextPtr)
|
||||
{
|
||||
}
|
||||
|
||||
std::unique_ptr<IObjectStorage> HDFSObjectStorage::cloneObjectStorage(const std::string &, const Poco::Util::AbstractConfiguration &, const std::string &, ContextPtr)
|
||||
{
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "HDFS object storage doesn't support cloning");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#endif
|
119
src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h
Normal file
119
src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h
Normal file
@ -0,0 +1,119 @@
|
||||
#pragma once
|
||||
#include <Common/config.h>
|
||||
|
||||
|
||||
#if USE_HDFS
|
||||
|
||||
#include <Disks/IDisk.h>
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <Storages/HDFS/HDFSCommon.h>
|
||||
#include <Core/UUID.h>
|
||||
#include <memory>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct HDFSObjectStorageSettings
|
||||
{
|
||||
|
||||
HDFSObjectStorageSettings() = default;
|
||||
|
||||
size_t min_bytes_for_seek;
|
||||
int objects_chunk_size_to_delete;
|
||||
int replication;
|
||||
|
||||
HDFSObjectStorageSettings(
|
||||
int min_bytes_for_seek_,
|
||||
int objects_chunk_size_to_delete_,
|
||||
int replication_)
|
||||
: min_bytes_for_seek(min_bytes_for_seek_)
|
||||
, objects_chunk_size_to_delete(objects_chunk_size_to_delete_)
|
||||
, replication(replication_)
|
||||
{}
|
||||
};
|
||||
|
||||
|
||||
class HDFSObjectStorage : public IObjectStorage
|
||||
{
|
||||
public:
|
||||
|
||||
using SettingsPtr = std::unique_ptr<HDFSObjectStorageSettings>;
|
||||
|
||||
HDFSObjectStorage(
|
||||
FileCachePtr && cache_,
|
||||
const String & hdfs_root_path_,
|
||||
SettingsPtr settings_,
|
||||
const Poco::Util::AbstractConfiguration & config_)
|
||||
: IObjectStorage(std::move(cache_))
|
||||
, config(config_)
|
||||
, hdfs_builder(createHDFSBuilder(hdfs_root_path_, config))
|
||||
, hdfs_fs(createHDFSFS(hdfs_builder.get()))
|
||||
, settings(std::move(settings_))
|
||||
{}
|
||||
|
||||
bool exists(const std::string & hdfs_uri) const override;
|
||||
|
||||
std::unique_ptr<SeekableReadBuffer> readObject( /// NOLINT
|
||||
const std::string & path,
|
||||
const ReadSettings & read_settings = ReadSettings{},
|
||||
std::optional<size_t> read_hint = {},
|
||||
std::optional<size_t> file_size = {}) const override;
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> readObjects( /// NOLINT
|
||||
const std::string & common_path_prefix,
|
||||
const BlobsPathToSize & blobs_to_read,
|
||||
const ReadSettings & read_settings = ReadSettings{},
|
||||
std::optional<size_t> read_hint = {},
|
||||
std::optional<size_t> file_size = {}) const override;
|
||||
|
||||
/// Open the file for write and return WriteBufferFromFileBase object.
|
||||
std::unique_ptr<WriteBufferFromFileBase> writeObject( /// NOLINT
|
||||
const std::string & path,
|
||||
WriteMode mode,
|
||||
std::optional<ObjectAttributes> attributes = {},
|
||||
FinalizeCallback && finalize_callback = {},
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
const WriteSettings & write_settings = {}) override;
|
||||
|
||||
void listPrefix(const std::string & path, BlobsPathToSize & children) const override;
|
||||
/// Remove file. Throws exception if file doesn't exists or it's a directory.
|
||||
void removeObject(const std::string & path) override;
|
||||
|
||||
void removeObjects(const std::vector<std::string> & paths) override;
|
||||
|
||||
void removeObjectIfExists(const std::string & path) override;
|
||||
|
||||
void removeObjectsIfExist(const std::vector<std::string> & paths) override;
|
||||
|
||||
ObjectMetadata getObjectMetadata(const std::string & path) const override;
|
||||
|
||||
void copyObject( /// NOLINT
|
||||
const std::string & object_from,
|
||||
const std::string & object_to,
|
||||
std::optional<ObjectAttributes> object_to_attributes = {}) override;
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
void startup() override;
|
||||
|
||||
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) override;
|
||||
|
||||
String getObjectsNamespace() const override { return ""; }
|
||||
|
||||
std::unique_ptr<IObjectStorage> cloneObjectStorage(const std::string & new_namespace, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) override;
|
||||
|
||||
private:
|
||||
const Poco::Util::AbstractConfiguration & config;
|
||||
|
||||
HDFSBuilderWrapper hdfs_builder;
|
||||
HDFSFSPtr hdfs_fs;
|
||||
|
||||
SettingsPtr settings;
|
||||
|
||||
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
54
src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp
Normal file
54
src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp
Normal file
@ -0,0 +1,54 @@
|
||||
#include <Disks/ObjectStorages/HDFS/HDFSObjectStorage.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageCommon.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorage.h>
|
||||
#include <Disks/DiskFactory.h>
|
||||
#include <Storages/HDFS/HDFSCommon.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
void registerDiskHDFS(DiskFactory & factory)
|
||||
{
|
||||
auto creator = [](const String & name,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const String & config_prefix,
|
||||
ContextPtr context_,
|
||||
const DisksMap & /*map*/) -> DiskPtr
|
||||
{
|
||||
String uri{config.getString(config_prefix + ".endpoint")};
|
||||
checkHDFSURL(uri);
|
||||
|
||||
if (uri.back() != '/')
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "HDFS path must ends with '/', but '{}' doesn't.", uri);
|
||||
|
||||
std::unique_ptr<HDFSObjectStorageSettings> settings = std::make_unique<HDFSObjectStorageSettings>(
|
||||
config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024),
|
||||
config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000),
|
||||
context_->getSettingsRef().hdfs_replication
|
||||
);
|
||||
/// FIXME Cache currently unsupported :(
|
||||
ObjectStoragePtr hdfs_storage = std::make_unique<HDFSObjectStorage>(nullptr, uri, std::move(settings), config);
|
||||
|
||||
auto metadata_disk = prepareForLocalMetadata(name, config, config_prefix, context_).second;
|
||||
uint64_t copy_thread_pool_size = config.getUInt(config_prefix + ".thread_pool_size", 16);
|
||||
|
||||
return std::make_shared<DiskObjectStorage>(
|
||||
name,
|
||||
uri,
|
||||
"DiskHDFS",
|
||||
metadata_disk,
|
||||
std::move(hdfs_storage),
|
||||
DiskType::HDFS,
|
||||
/* send_metadata = */ false,
|
||||
copy_thread_pool_size);
|
||||
};
|
||||
|
||||
factory.registerDiskType("hdfs", creator);
|
||||
}
|
||||
|
||||
}
|
48
src/Disks/ObjectStorages/IObjectStorage.cpp
Normal file
48
src/Disks/ObjectStorages/IObjectStorage.cpp
Normal file
@ -0,0 +1,48 @@
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <Disks/IO/ThreadPoolRemoteFSReader.h>
|
||||
#include <IO/copyData.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
AsynchronousReaderPtr IObjectStorage::getThreadPoolReader()
|
||||
{
|
||||
constexpr size_t pool_size = 50;
|
||||
constexpr size_t queue_size = 1000000;
|
||||
static AsynchronousReaderPtr reader = std::make_shared<ThreadPoolRemoteFSReader>(pool_size, queue_size);
|
||||
return reader;
|
||||
}
|
||||
|
||||
ThreadPool & IObjectStorage::getThreadPoolWriter()
|
||||
{
|
||||
constexpr size_t pool_size = 100;
|
||||
constexpr size_t queue_size = 1000000;
|
||||
static ThreadPool writer(pool_size, pool_size, queue_size);
|
||||
return writer;
|
||||
}
|
||||
|
||||
std::string IObjectStorage::getCacheBasePath() const
|
||||
{
|
||||
return cache ? cache->getBasePath() : "";
|
||||
}
|
||||
|
||||
void IObjectStorage::removeFromCache(const std::string & path)
|
||||
{
|
||||
if (cache)
|
||||
{
|
||||
auto key = cache->hash(path);
|
||||
cache->remove(key);
|
||||
}
|
||||
}
|
||||
|
||||
void IObjectStorage::copyObjectToAnotherObjectStorage(const std::string & object_from, const std::string & object_to, IObjectStorage & object_storage_to, std::optional<ObjectAttributes> object_to_attributes) // NOLINT
|
||||
{
|
||||
if (&object_storage_to == this)
|
||||
copyObject(object_from, object_to, object_to_attributes);
|
||||
|
||||
auto in = readObject(object_from);
|
||||
auto out = object_storage_to.writeObject(object_to, WriteMode::Rewrite);
|
||||
copyData(*in, *out);
|
||||
out->finalize();
|
||||
}
|
||||
|
||||
}
|
159
src/Disks/ObjectStorages/IObjectStorage.h
Normal file
159
src/Disks/ObjectStorages/IObjectStorage.h
Normal file
@ -0,0 +1,159 @@
|
||||
#pragma once
|
||||
|
||||
#include <filesystem>
|
||||
#include <string>
|
||||
#include <map>
|
||||
#include <optional>
|
||||
|
||||
#include <Poco/Timestamp.h>
|
||||
#include <Core/Defines.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <IO/ReadSettings.h>
|
||||
#include <IO/WriteSettings.h>
|
||||
|
||||
#include <Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Common/FileCache.h>
|
||||
#include <Disks/WriteMode.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ReadBufferFromFileBase;
|
||||
class WriteBufferFromFileBase;
|
||||
|
||||
using ObjectAttributes = std::map<std::string, std::string>;
|
||||
|
||||
/// Path to blob with it's size
|
||||
struct BlobPathWithSize
|
||||
{
|
||||
std::string relative_path;
|
||||
uint64_t bytes_size;
|
||||
|
||||
BlobPathWithSize() = default;
|
||||
BlobPathWithSize(const BlobPathWithSize & other) = default;
|
||||
|
||||
BlobPathWithSize(const std::string & relative_path_, uint64_t bytes_size_)
|
||||
: relative_path(relative_path_)
|
||||
, bytes_size(bytes_size_)
|
||||
{}
|
||||
};
|
||||
|
||||
/// List of blobs with their sizes
|
||||
using BlobsPathToSize = std::vector<BlobPathWithSize>;
|
||||
|
||||
struct ObjectMetadata
|
||||
{
|
||||
uint64_t size_bytes;
|
||||
std::optional<Poco::Timestamp> last_modified;
|
||||
std::optional<ObjectAttributes> attributes;
|
||||
};
|
||||
|
||||
using FinalizeCallback = std::function<void(size_t bytes_count)>;
|
||||
|
||||
/// Base class for all object storages which implement some subset of ordinary filesystem operations.
|
||||
///
|
||||
/// Examples of object storages are S3, Azure Blob Storage, HDFS.
|
||||
class IObjectStorage
|
||||
{
|
||||
public:
|
||||
explicit IObjectStorage(FileCachePtr && cache_)
|
||||
: cache(std::move(cache_))
|
||||
{}
|
||||
|
||||
/// Path exists or not
|
||||
virtual bool exists(const std::string & path) const = 0;
|
||||
|
||||
/// List on prefix, return children with their sizes.
|
||||
virtual void listPrefix(const std::string & path, BlobsPathToSize & children) const = 0;
|
||||
|
||||
/// Get object metadata if supported. It should be possible to receive
|
||||
/// at least size of object
|
||||
virtual ObjectMetadata getObjectMetadata(const std::string & path) const = 0;
|
||||
|
||||
/// Read single path from object storage
|
||||
virtual std::unique_ptr<SeekableReadBuffer> readObject( /// NOLINT
|
||||
const std::string & path,
|
||||
const ReadSettings & read_settings = ReadSettings{},
|
||||
std::optional<size_t> read_hint = {},
|
||||
std::optional<size_t> file_size = {}) const = 0;
|
||||
|
||||
/// Read multiple objects with common prefix
|
||||
virtual std::unique_ptr<ReadBufferFromFileBase> readObjects( /// NOLINT
|
||||
const std::string & common_path_prefix,
|
||||
const BlobsPathToSize & blobs_to_read,
|
||||
const ReadSettings & read_settings = ReadSettings{},
|
||||
std::optional<size_t> read_hint = {},
|
||||
std::optional<size_t> file_size = {}) const = 0;
|
||||
|
||||
/// Open the file for write and return WriteBufferFromFileBase object.
|
||||
virtual std::unique_ptr<WriteBufferFromFileBase> writeObject( /// NOLINT
|
||||
const std::string & path,
|
||||
WriteMode mode,
|
||||
std::optional<ObjectAttributes> attributes = {},
|
||||
FinalizeCallback && finalize_callback = {},
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
const WriteSettings & write_settings = {}) = 0;
|
||||
|
||||
/// Remove object. Throws exception if object doesn't exists.
|
||||
virtual void removeObject(const std::string & path) = 0;
|
||||
|
||||
/// Remove multiple objects. Some object storages can do batch remove in a more
|
||||
/// optimal way.
|
||||
virtual void removeObjects(const std::vector<std::string> & paths) = 0;
|
||||
|
||||
/// Remove object on path if exists
|
||||
virtual void removeObjectIfExists(const std::string & path) = 0;
|
||||
|
||||
/// Remove objects on path if exists
|
||||
virtual void removeObjectsIfExist(const std::vector<std::string> & paths) = 0;
|
||||
|
||||
/// Copy object with different attributes if required
|
||||
virtual void copyObject( /// NOLINT
|
||||
const std::string & object_from,
|
||||
const std::string & object_to,
|
||||
std::optional<ObjectAttributes> object_to_attributes = {}) = 0;
|
||||
|
||||
/// Copy object to another instance of object storage
|
||||
/// by default just read the object from source object storage and write
|
||||
/// to destination through buffers.
|
||||
virtual void copyObjectToAnotherObjectStorage( /// NOLINT
|
||||
const std::string & object_from,
|
||||
const std::string & object_to,
|
||||
IObjectStorage & object_storage_to,
|
||||
std::optional<ObjectAttributes> object_to_attributes = {});
|
||||
|
||||
virtual ~IObjectStorage() = default;
|
||||
|
||||
/// Path to directory with objects cache
|
||||
std::string getCacheBasePath() const;
|
||||
|
||||
static AsynchronousReaderPtr getThreadPoolReader();
|
||||
|
||||
static ThreadPool & getThreadPoolWriter();
|
||||
|
||||
virtual void shutdown() = 0;
|
||||
|
||||
virtual void startup() = 0;
|
||||
|
||||
void removeFromCache(const std::string & path);
|
||||
|
||||
/// Apply new settings, in most cases reiniatilize client and some other staff
|
||||
virtual void applyNewSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) = 0;
|
||||
|
||||
/// Sometimes object storages have something similar to chroot or namespace, for example
|
||||
/// buckets in S3. If object storage doesn't have any namepaces return empty string.
|
||||
virtual String getObjectsNamespace() const = 0;
|
||||
|
||||
/// FIXME: confusing function required for a very specific case. Create new instance of object storage
|
||||
/// in different namespace.
|
||||
virtual std::unique_ptr<IObjectStorage> cloneObjectStorage(const std::string & new_namespace, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) = 0;
|
||||
|
||||
protected:
|
||||
FileCachePtr cache;
|
||||
};
|
||||
|
||||
using ObjectStoragePtr = std::unique_ptr<IObjectStorage>;
|
||||
|
||||
}
|
502
src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp
Normal file
502
src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp
Normal file
@ -0,0 +1,502 @@
|
||||
#include <Disks/ObjectStorages/S3/S3ObjectStorage.h>
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageCommon.h>
|
||||
#include <Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/ReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/WriteIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/ThreadPoolRemoteFSReader.h>
|
||||
#include <IO/WriteBufferFromS3.h>
|
||||
#include <IO/ReadBufferFromS3.h>
|
||||
#include <IO/SeekAvoidingReadBuffer.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
#include <Disks/ObjectStorages/S3/diskSettings.h>
|
||||
|
||||
#include <aws/s3/model/CopyObjectRequest.h>
|
||||
#include <aws/s3/model/ListObjectsV2Request.h>
|
||||
#include <aws/s3/model/HeadObjectRequest.h>
|
||||
#include <aws/s3/model/DeleteObjectsRequest.h>
|
||||
#include <aws/s3/model/CreateMultipartUploadRequest.h>
|
||||
#include <aws/s3/model/CompleteMultipartUploadRequest.h>
|
||||
#include <aws/s3/model/UploadPartCopyRequest.h>
|
||||
#include <aws/s3/model/AbortMultipartUploadRequest.h>
|
||||
|
||||
#include <Common/FileCache.h>
|
||||
#include <Common/FileCacheFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int S3_ERROR;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
template <typename Result, typename Error>
|
||||
void throwIfError(Aws::Utils::Outcome<Result, Error> & response)
|
||||
{
|
||||
if (!response.IsSuccess())
|
||||
{
|
||||
const auto & err = response.GetError();
|
||||
throw Exception(std::to_string(static_cast<int>(err.GetErrorType())) + ": " + err.GetMessage(), ErrorCodes::S3_ERROR);
|
||||
}
|
||||
}
|
||||
|
||||
template <typename Result, typename Error>
|
||||
void throwIfError(const Aws::Utils::Outcome<Result, Error> & response)
|
||||
{
|
||||
if (!response.IsSuccess())
|
||||
{
|
||||
const auto & err = response.GetError();
|
||||
throw Exception(err.GetMessage(), static_cast<int>(err.GetErrorType()));
|
||||
}
|
||||
}
|
||||
|
||||
template <typename Result, typename Error>
|
||||
void logIfError(const Aws::Utils::Outcome<Result, Error> & response, std::function<String()> && msg)
|
||||
{
|
||||
try
|
||||
{
|
||||
throwIfError(response);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__, msg());
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
Aws::S3::Model::HeadObjectOutcome S3ObjectStorage::requestObjectHeadData(const std::string & bucket_from, const std::string & key) const
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
Aws::S3::Model::HeadObjectRequest request;
|
||||
request.SetBucket(bucket_from);
|
||||
request.SetKey(key);
|
||||
|
||||
return client_ptr->HeadObject(request);
|
||||
}
|
||||
|
||||
bool S3ObjectStorage::exists(const std::string & path) const
|
||||
{
|
||||
auto object_head = requestObjectHeadData(bucket, path);
|
||||
if (!object_head.IsSuccess())
|
||||
{
|
||||
if (object_head.GetError().GetErrorType() == Aws::S3::S3Errors::RESOURCE_NOT_FOUND)
|
||||
return false;
|
||||
|
||||
throwIfError(object_head);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> S3ObjectStorage::readObjects( /// NOLINT
|
||||
const std::string & common_path_prefix,
|
||||
const BlobsPathToSize & blobs_to_read,
|
||||
const ReadSettings & read_settings,
|
||||
std::optional<size_t>,
|
||||
std::optional<size_t>) const
|
||||
{
|
||||
|
||||
ReadSettings disk_read_settings{read_settings};
|
||||
if (cache)
|
||||
{
|
||||
if (IFileCache::isReadOnly())
|
||||
disk_read_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = true;
|
||||
|
||||
disk_read_settings.remote_fs_cache = cache;
|
||||
}
|
||||
|
||||
auto settings_ptr = s3_settings.get();
|
||||
|
||||
auto s3_impl = std::make_unique<ReadBufferFromS3Gather>(
|
||||
client.get(), bucket, version_id, common_path_prefix, blobs_to_read,
|
||||
settings_ptr->s3_settings.max_single_read_retries, disk_read_settings);
|
||||
|
||||
if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)
|
||||
{
|
||||
auto reader = getThreadPoolReader();
|
||||
return std::make_unique<AsynchronousReadIndirectBufferFromRemoteFS>(reader, disk_read_settings, std::move(s3_impl));
|
||||
}
|
||||
else
|
||||
{
|
||||
auto buf = std::make_unique<ReadIndirectBufferFromRemoteFS>(std::move(s3_impl));
|
||||
return std::make_unique<SeekAvoidingReadBuffer>(std::move(buf), settings_ptr->min_bytes_for_seek);
|
||||
}
|
||||
}
|
||||
|
||||
std::unique_ptr<SeekableReadBuffer> S3ObjectStorage::readObject( /// NOLINT
|
||||
const std::string & path,
|
||||
const ReadSettings & read_settings,
|
||||
std::optional<size_t>,
|
||||
std::optional<size_t>) const
|
||||
{
|
||||
auto settings_ptr = s3_settings.get();
|
||||
ReadSettings disk_read_settings{read_settings};
|
||||
if (cache)
|
||||
{
|
||||
if (IFileCache::isReadOnly())
|
||||
disk_read_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = true;
|
||||
|
||||
disk_read_settings.remote_fs_cache = cache;
|
||||
}
|
||||
|
||||
return std::make_unique<ReadBufferFromS3>(client.get(), bucket, path, version_id, settings_ptr->s3_settings.max_single_read_retries, disk_read_settings);
|
||||
}
|
||||
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> S3ObjectStorage::writeObject( /// NOLINT
|
||||
const std::string & path,
|
||||
WriteMode mode, // S3 doesn't support append, only rewrite
|
||||
std::optional<ObjectAttributes> attributes,
|
||||
FinalizeCallback && finalize_callback,
|
||||
size_t buf_size,
|
||||
const WriteSettings & write_settings)
|
||||
{
|
||||
if (mode != WriteMode::Rewrite)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3 doesn't support append to files");
|
||||
|
||||
bool cache_on_write = cache
|
||||
&& fs::path(path).extension() != ".tmp"
|
||||
&& write_settings.enable_filesystem_cache_on_write_operations
|
||||
&& FileCacheFactory::instance().getSettings(getCacheBasePath()).cache_on_write_operations;
|
||||
|
||||
auto settings_ptr = s3_settings.get();
|
||||
auto s3_buffer = std::make_unique<WriteBufferFromS3>(
|
||||
client.get(),
|
||||
bucket,
|
||||
path,
|
||||
settings_ptr->s3_settings,
|
||||
attributes,
|
||||
buf_size, threadPoolCallbackRunner(getThreadPoolWriter()),
|
||||
cache_on_write ? cache : nullptr);
|
||||
|
||||
return std::make_unique<WriteIndirectBufferFromRemoteFS>(std::move(s3_buffer), std::move(finalize_callback), path);
|
||||
}
|
||||
|
||||
void S3ObjectStorage::listPrefix(const std::string & path, BlobsPathToSize & children) const
|
||||
{
|
||||
auto settings_ptr = s3_settings.get();
|
||||
auto client_ptr = client.get();
|
||||
|
||||
Aws::S3::Model::ListObjectsV2Request request;
|
||||
request.SetBucket(bucket);
|
||||
request.SetPrefix(path);
|
||||
request.SetMaxKeys(settings_ptr->list_object_keys_size);
|
||||
|
||||
Aws::S3::Model::ListObjectsV2Outcome outcome;
|
||||
do
|
||||
{
|
||||
outcome = client_ptr->ListObjectsV2(request);
|
||||
throwIfError(outcome);
|
||||
|
||||
auto result = outcome.GetResult();
|
||||
auto objects = result.GetContents();
|
||||
|
||||
if (objects.empty())
|
||||
break;
|
||||
|
||||
for (const auto & object : objects)
|
||||
children.emplace_back(object.GetKey(), object.GetSize());
|
||||
|
||||
request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken());
|
||||
} while (outcome.GetResult().GetIsTruncated());
|
||||
}
|
||||
|
||||
void S3ObjectStorage::removeObject(const std::string & path)
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
Aws::S3::Model::ObjectIdentifier obj;
|
||||
obj.SetKey(path);
|
||||
|
||||
Aws::S3::Model::Delete delkeys;
|
||||
delkeys.SetObjects({obj});
|
||||
|
||||
Aws::S3::Model::DeleteObjectsRequest request;
|
||||
request.SetBucket(bucket);
|
||||
request.SetDelete(delkeys);
|
||||
auto outcome = client_ptr->DeleteObjects(request);
|
||||
|
||||
throwIfError(outcome);
|
||||
}
|
||||
|
||||
void S3ObjectStorage::removeObjects(const std::vector<std::string> & paths)
|
||||
{
|
||||
if (paths.empty())
|
||||
return;
|
||||
|
||||
auto client_ptr = client.get();
|
||||
auto settings_ptr = s3_settings.get();
|
||||
|
||||
size_t chunk_size_limit = settings_ptr->objects_chunk_size_to_delete;
|
||||
size_t current_position = 0;
|
||||
|
||||
while (current_position < paths.size())
|
||||
{
|
||||
std::vector<Aws::S3::Model::ObjectIdentifier> current_chunk;
|
||||
String keys;
|
||||
for (; current_position < paths.size() && current_chunk.size() < chunk_size_limit; ++current_position)
|
||||
{
|
||||
Aws::S3::Model::ObjectIdentifier obj;
|
||||
obj.SetKey(paths[current_position]);
|
||||
current_chunk.push_back(obj);
|
||||
|
||||
if (!keys.empty())
|
||||
keys += ", ";
|
||||
keys += paths[current_position];
|
||||
}
|
||||
|
||||
Aws::S3::Model::Delete delkeys;
|
||||
delkeys.SetObjects(current_chunk);
|
||||
Aws::S3::Model::DeleteObjectsRequest request;
|
||||
request.SetBucket(bucket);
|
||||
request.SetDelete(delkeys);
|
||||
auto outcome = client_ptr->DeleteObjects(request);
|
||||
throwIfError(outcome);
|
||||
}
|
||||
}
|
||||
|
||||
void S3ObjectStorage::removeObjectIfExists(const std::string & path)
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
Aws::S3::Model::ObjectIdentifier obj;
|
||||
obj.SetKey(path);
|
||||
|
||||
Aws::S3::Model::Delete delkeys;
|
||||
delkeys.SetObjects({obj});
|
||||
|
||||
Aws::S3::Model::DeleteObjectsRequest request;
|
||||
request.SetBucket(bucket);
|
||||
request.SetDelete(delkeys);
|
||||
auto outcome = client_ptr->DeleteObjects(request);
|
||||
if (!outcome.IsSuccess() && outcome.GetError().GetErrorType() != Aws::S3::S3Errors::RESOURCE_NOT_FOUND)
|
||||
throwIfError(outcome);
|
||||
}
|
||||
|
||||
void S3ObjectStorage::removeObjectsIfExist(const std::vector<std::string> & paths)
|
||||
{
|
||||
if (paths.empty())
|
||||
return;
|
||||
|
||||
auto client_ptr = client.get();
|
||||
auto settings_ptr = s3_settings.get();
|
||||
|
||||
|
||||
size_t chunk_size_limit = settings_ptr->objects_chunk_size_to_delete;
|
||||
size_t current_position = 0;
|
||||
|
||||
while (current_position < paths.size())
|
||||
{
|
||||
std::vector<Aws::S3::Model::ObjectIdentifier> current_chunk;
|
||||
String keys;
|
||||
for (; current_position < paths.size() && current_chunk.size() < chunk_size_limit; ++current_position)
|
||||
{
|
||||
Aws::S3::Model::ObjectIdentifier obj;
|
||||
obj.SetKey(paths[current_position]);
|
||||
current_chunk.push_back(obj);
|
||||
|
||||
if (!keys.empty())
|
||||
keys += ", ";
|
||||
keys += paths[current_position];
|
||||
}
|
||||
|
||||
Aws::S3::Model::Delete delkeys;
|
||||
delkeys.SetObjects(current_chunk);
|
||||
Aws::S3::Model::DeleteObjectsRequest request;
|
||||
request.SetBucket(bucket);
|
||||
request.SetDelete(delkeys);
|
||||
auto outcome = client_ptr->DeleteObjects(request);
|
||||
if (!outcome.IsSuccess() && outcome.GetError().GetErrorType() != Aws::S3::S3Errors::RESOURCE_NOT_FOUND)
|
||||
throwIfError(outcome);
|
||||
}
|
||||
}
|
||||
|
||||
ObjectMetadata S3ObjectStorage::getObjectMetadata(const std::string & path) const
|
||||
{
|
||||
ObjectMetadata result;
|
||||
|
||||
auto object_head = requestObjectHeadData(bucket, path);
|
||||
throwIfError(object_head);
|
||||
|
||||
auto & object_head_result = object_head.GetResult();
|
||||
result.size_bytes = object_head_result.GetContentLength();
|
||||
result.last_modified = object_head_result.GetLastModified().Millis();
|
||||
result.attributes = object_head_result.GetMetadata();
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
void S3ObjectStorage::copyObjectToAnotherObjectStorage(const std::string & object_from, const std::string & object_to, IObjectStorage & object_storage_to, std::optional<ObjectAttributes> object_to_attributes) // NOLINT
|
||||
{
|
||||
/// Shortcut for S3
|
||||
if (auto * dest_s3 = dynamic_cast<S3ObjectStorage * >(&object_storage_to); dest_s3 != nullptr)
|
||||
copyObjectImpl(bucket, object_from, dest_s3->bucket, object_to, {}, object_to_attributes);
|
||||
else
|
||||
IObjectStorage::copyObjectToAnotherObjectStorage(object_from, object_to, object_storage_to, object_to_attributes);
|
||||
}
|
||||
|
||||
void S3ObjectStorage::copyObjectImpl(const String & src_bucket, const String & src_key, const String & dst_bucket, const String & dst_key,
|
||||
std::optional<Aws::S3::Model::HeadObjectResult> head,
|
||||
std::optional<ObjectAttributes> metadata) const
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
Aws::S3::Model::CopyObjectRequest request;
|
||||
request.SetCopySource(src_bucket + "/" + src_key);
|
||||
request.SetBucket(dst_bucket);
|
||||
request.SetKey(dst_key);
|
||||
if (metadata)
|
||||
{
|
||||
request.SetMetadata(*metadata);
|
||||
request.SetMetadataDirective(Aws::S3::Model::MetadataDirective::REPLACE);
|
||||
}
|
||||
|
||||
auto outcome = client_ptr->CopyObject(request);
|
||||
|
||||
if (!outcome.IsSuccess() && outcome.GetError().GetExceptionName() == "EntityTooLarge")
|
||||
{ // Can't come here with MinIO, MinIO allows single part upload for large objects.
|
||||
copyObjectMultipartImpl(src_bucket, src_key, dst_bucket, dst_key, head, metadata);
|
||||
return;
|
||||
}
|
||||
|
||||
throwIfError(outcome);
|
||||
}
|
||||
|
||||
void S3ObjectStorage::copyObjectMultipartImpl(const String & src_bucket, const String & src_key, const String & dst_bucket, const String & dst_key,
|
||||
std::optional<Aws::S3::Model::HeadObjectResult> head,
|
||||
std::optional<ObjectAttributes> metadata) const
|
||||
{
|
||||
if (!head)
|
||||
head = requestObjectHeadData(src_bucket, src_key).GetResult();
|
||||
|
||||
auto settings_ptr = s3_settings.get();
|
||||
auto client_ptr = client.get();
|
||||
size_t size = head->GetContentLength();
|
||||
|
||||
String multipart_upload_id;
|
||||
|
||||
{
|
||||
Aws::S3::Model::CreateMultipartUploadRequest request;
|
||||
request.SetBucket(dst_bucket);
|
||||
request.SetKey(dst_key);
|
||||
if (metadata)
|
||||
request.SetMetadata(*metadata);
|
||||
|
||||
auto outcome = client_ptr->CreateMultipartUpload(request);
|
||||
|
||||
throwIfError(outcome);
|
||||
|
||||
multipart_upload_id = outcome.GetResult().GetUploadId();
|
||||
}
|
||||
|
||||
std::vector<String> part_tags;
|
||||
|
||||
size_t upload_part_size = settings_ptr->s3_settings.min_upload_part_size;
|
||||
for (size_t position = 0, part_number = 1; position < size; ++part_number, position += upload_part_size)
|
||||
{
|
||||
Aws::S3::Model::UploadPartCopyRequest part_request;
|
||||
part_request.SetCopySource(src_bucket + "/" + src_key);
|
||||
part_request.SetBucket(dst_bucket);
|
||||
part_request.SetKey(dst_key);
|
||||
part_request.SetUploadId(multipart_upload_id);
|
||||
part_request.SetPartNumber(part_number);
|
||||
part_request.SetCopySourceRange(fmt::format("bytes={}-{}", position, std::min(size, position + upload_part_size) - 1));
|
||||
|
||||
auto outcome = client_ptr->UploadPartCopy(part_request);
|
||||
if (!outcome.IsSuccess())
|
||||
{
|
||||
Aws::S3::Model::AbortMultipartUploadRequest abort_request;
|
||||
abort_request.SetBucket(dst_bucket);
|
||||
abort_request.SetKey(dst_key);
|
||||
abort_request.SetUploadId(multipart_upload_id);
|
||||
client_ptr->AbortMultipartUpload(abort_request);
|
||||
// In error case we throw exception later with first error from UploadPartCopy
|
||||
}
|
||||
throwIfError(outcome);
|
||||
|
||||
auto etag = outcome.GetResult().GetCopyPartResult().GetETag();
|
||||
part_tags.push_back(etag);
|
||||
}
|
||||
|
||||
{
|
||||
Aws::S3::Model::CompleteMultipartUploadRequest req;
|
||||
req.SetBucket(dst_bucket);
|
||||
req.SetKey(dst_key);
|
||||
req.SetUploadId(multipart_upload_id);
|
||||
|
||||
Aws::S3::Model::CompletedMultipartUpload multipart_upload;
|
||||
for (size_t i = 0; i < part_tags.size(); ++i)
|
||||
{
|
||||
Aws::S3::Model::CompletedPart part;
|
||||
multipart_upload.AddParts(part.WithETag(part_tags[i]).WithPartNumber(i + 1));
|
||||
}
|
||||
|
||||
req.SetMultipartUpload(multipart_upload);
|
||||
|
||||
auto outcome = client_ptr->CompleteMultipartUpload(req);
|
||||
|
||||
throwIfError(outcome);
|
||||
}
|
||||
}
|
||||
|
||||
void S3ObjectStorage::copyObject(const std::string & object_from, const std::string & object_to, std::optional<ObjectAttributes> object_to_attributes) // NOLINT
|
||||
{
|
||||
auto head = requestObjectHeadData(bucket, object_from).GetResult();
|
||||
if (head.GetContentLength() >= static_cast<int64_t>(5UL * 1024 * 1024 * 1024))
|
||||
copyObjectMultipartImpl(bucket, object_from, bucket, object_to, head, object_to_attributes);
|
||||
else
|
||||
copyObjectImpl(bucket, object_from, bucket, object_to, head, object_to_attributes);
|
||||
}
|
||||
|
||||
void S3ObjectStorage::setNewSettings(std::unique_ptr<S3ObjectStorageSettings> && s3_settings_)
|
||||
{
|
||||
s3_settings.set(std::move(s3_settings_));
|
||||
}
|
||||
|
||||
void S3ObjectStorage::setNewClient(std::unique_ptr<Aws::S3::S3Client> && client_)
|
||||
{
|
||||
client.set(std::move(client_));
|
||||
}
|
||||
|
||||
void S3ObjectStorage::shutdown()
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
/// This call stops any next retry attempts for ongoing S3 requests.
|
||||
/// If S3 request is failed and the method below is executed S3 client immediately returns the last failed S3 request outcome.
|
||||
/// If S3 is healthy nothing wrong will be happened and S3 requests will be processed in a regular way without errors.
|
||||
/// This should significantly speed up shutdown process if S3 is unhealthy.
|
||||
const_cast<Aws::S3::S3Client &>(*client_ptr).DisableRequestProcessing();
|
||||
}
|
||||
|
||||
void S3ObjectStorage::startup()
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
|
||||
/// Need to be enabled if it was disabled during shutdown() call.
|
||||
const_cast<Aws::S3::S3Client &>(*client_ptr).EnableRequestProcessing();
|
||||
}
|
||||
|
||||
void S3ObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context)
|
||||
{
|
||||
s3_settings.set(getSettings(config, config_prefix, context));
|
||||
client.set(getClient(config, config_prefix, context));
|
||||
}
|
||||
|
||||
std::unique_ptr<IObjectStorage> S3ObjectStorage::cloneObjectStorage(const std::string & new_namespace, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context)
|
||||
{
|
||||
return std::make_unique<S3ObjectStorage>(
|
||||
nullptr, getClient(config, config_prefix, context),
|
||||
getSettings(config, config_prefix, context),
|
||||
version_id, new_namespace);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
#endif
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user