Merge remote-tracking branch 'upstream/master' into file_descriptor

This commit is contained in:
wuxiaobai24 2022-04-28 22:23:25 +08:00
commit 82a053dbb5
212 changed files with 6428 additions and 1217 deletions

2
contrib/NuRaft vendored

@ -1 +1 @@
Subproject commit 1707a7572aa66ec5d0a2dbe2bf5effa3352e6b2d
Subproject commit 0e1ac401b7c41a5042be2385477332dbeb7104cd

View File

@ -159,6 +159,14 @@ We maintain full backward and forward compatibility for the server TCP protocol:
For most external applications, we recommend using the HTTP interface because it is simple and easy to use. The TCP protocol is more tightly linked to internal data structures: it uses an internal format for passing blocks of data, and it uses custom framing for compressed data. We havent released a C library for that protocol because it requires linking most of the ClickHouse codebase, which is not practical.
:::
## Configuration {#configuration}
ClickHouse Server is based on POCO C++ Libraries and uses `Poco::Util::AbstractConfiguration` to represent it's configuration. Configuration is held by `Poco::Util::ServerApplication` class inherited by `DaemonBase` class, which in turn is inherited by `DB::Server` class, implementing clickhouse-server itself. So config can be accessed by `ServerApplication::config()` method.
Config is read from multiple files (in XML or YAML format) and merged into single `AbstractConfiguration` by `ConfigProcessor` class. Configuration is loaded at server startup and can be reloaded later if one of config files is updated, removed or added. `ConfigReloader` class is responsible for periodic monitoring of these changes and reload procedure as well. `SYSTEM RELOAD CONFIG` query also triggers config to be reloaded.
For queries and subsystems other than `Server` config is accessible using `Context::getConfigRef()` method. Every subsystem that is capable of reloading it's config without server restart should register itself in reload callback in `Server::main()` method. Note that if newer config has an error, most subsystems will ignore new config, log warning messages and keep working with previously loaded config. Due to the nature of `AbstractConfiguration` it is not possible to pass reference to specific section, so `String config_prefix` is usually used instead.
## Distributed Query Execution {#distributed-query-execution}
Servers in a cluster setup are mostly independent. You can create a `Distributed` table on one or all servers in a cluster. The `Distributed` table does not store data itself it only provides a “view” to all local tables on multiple nodes of a cluster. When you SELECT from a `Distributed` table, it rewrites that query, chooses remote nodes according to load balancing settings, and sends the query to them. The `Distributed` table requests remote servers to process a query just up to a stage where intermediate results from different servers can be merged. Then it receives the intermediate results and merges them. The distributed table tries to distribute as much work as possible to remote servers and does not send much intermediate data over the network.

View File

@ -49,6 +49,7 @@ The supported formats are:
| [PrettyCompactMonoBlock](#prettycompactmonoblock) | ✗ | ✔ |
| [PrettyNoEscapes](#prettynoescapes) | ✗ | ✔ |
| [PrettySpace](#prettyspace) | ✗ | ✔ |
| [Prometheus](#prometheus) | ✗ | ✔ |
| [Protobuf](#protobuf) | ✔ | ✔ |
| [ProtobufSingle](#protobufsingle) | ✔ | ✔ |
| [Avro](#data-format-avro) | ✔ | ✔ |
@ -1162,6 +1163,76 @@ You can select data from a ClickHouse table and save them into some file in the
``` bash
$ clickhouse-client --query = "SELECT * FROM test.hits FORMAT CapnProto SETTINGS format_schema = 'schema:Message'"
```
## Prometheus {#prometheus}
Expose metrics in [Prometheus text-based exposition format](https://prometheus.io/docs/instrumenting/exposition_formats/#text-based-format).
The output table should have a proper structure.
Columns `name` ([String](../sql-reference/data-types/string.md)) and `value` (number) are required.
Rows may optionally contain `help` ([String](../sql-reference/data-types/string.md)) and `timestamp` (number).
Column `type` ([String](../sql-reference/data-types/string.md)) is either `counter`, `gauge`, `histogram`, `summary`, `untyped` or empty.
Each metric value may also have some `labels` ([Map(String, String)](../sql-reference/data-types/map.md)).
Several consequent rows may refer to the one metric with different lables. The table should be sorted by metric name (e.g., with `ORDER BY name`).
There's special requirements for labels for `histogram` and `summary`, see [Prometheus doc](https://prometheus.io/docs/instrumenting/exposition_formats/#histograms-and-summaries) for the details. Special rules applied to row with labels `{'count':''}` and `{'sum':''}`, they'll be convered to `<metric_name>_count` and `<metric_name>_sum` respectively.
**Example:**
```
┌─name────────────────────────────────┬─type──────┬─help──────────────────────────────────────┬─labels─────────────────────────┬────value─┬─────timestamp─┐
│ http_request_duration_seconds │ histogram │ A histogram of the request duration. │ {'le':'0.05'} │ 24054 │ 0 │
│ http_request_duration_seconds │ histogram │ │ {'le':'0.1'} │ 33444 │ 0 │
│ http_request_duration_seconds │ histogram │ │ {'le':'0.2'} │ 100392 │ 0 │
│ http_request_duration_seconds │ histogram │ │ {'le':'0.5'} │ 129389 │ 0 │
│ http_request_duration_seconds │ histogram │ │ {'le':'1'} │ 133988 │ 0 │
│ http_request_duration_seconds │ histogram │ │ {'le':'+Inf'} │ 144320 │ 0 │
│ http_request_duration_seconds │ histogram │ │ {'sum':''} │ 53423 │ 0 │
│ http_requests_total │ counter │ Total number of HTTP requests │ {'method':'post','code':'200'} │ 1027 │ 1395066363000 │
│ http_requests_total │ counter │ │ {'method':'post','code':'400'} │ 3 │ 1395066363000 │
│ metric_without_timestamp_and_labels │ │ │ {} │ 12.47 │ 0 │
│ rpc_duration_seconds │ summary │ A summary of the RPC duration in seconds. │ {'quantile':'0.01'} │ 3102 │ 0 │
│ rpc_duration_seconds │ summary │ │ {'quantile':'0.05'} │ 3272 │ 0 │
│ rpc_duration_seconds │ summary │ │ {'quantile':'0.5'} │ 4773 │ 0 │
│ rpc_duration_seconds │ summary │ │ {'quantile':'0.9'} │ 9001 │ 0 │
│ rpc_duration_seconds │ summary │ │ {'quantile':'0.99'} │ 76656 │ 0 │
│ rpc_duration_seconds │ summary │ │ {'count':''} │ 2693 │ 0 │
│ rpc_duration_seconds │ summary │ │ {'sum':''} │ 17560473 │ 0 │
│ something_weird │ │ │ {'problem':'division by zero'} │ inf │ -3982045 │
└─────────────────────────────────────┴───────────┴───────────────────────────────────────────┴────────────────────────────────┴──────────┴───────────────┘
```
Will be formatted as:
```
# HELP http_request_duration_seconds A histogram of the request duration.
# TYPE http_request_duration_seconds histogram
http_request_duration_seconds_bucket{le="0.05"} 24054
http_request_duration_seconds_bucket{le="0.1"} 33444
http_request_duration_seconds_bucket{le="0.5"} 129389
http_request_duration_seconds_bucket{le="1"} 133988
http_request_duration_seconds_bucket{le="+Inf"} 144320
http_request_duration_seconds_sum 53423
http_request_duration_seconds_count 144320
# HELP http_requests_total Total number of HTTP requests
# TYPE http_requests_total counter
http_requests_total{code="200",method="post"} 1027 1395066363000
http_requests_total{code="400",method="post"} 3 1395066363000
metric_without_timestamp_and_labels 12.47
# HELP rpc_duration_seconds A summary of the RPC duration in seconds.
# TYPE rpc_duration_seconds summary
rpc_duration_seconds{quantile="0.01"} 3102
rpc_duration_seconds{quantile="0.05"} 3272
rpc_duration_seconds{quantile="0.5"} 4773
rpc_duration_seconds{quantile="0.9"} 9001
rpc_duration_seconds{quantile="0.99"} 76656
rpc_duration_seconds_sum 17560473
rpc_duration_seconds_count 2693
something_weird{problem="division by zero"} +Inf -3982045
```
## Protobuf {#protobuf}

View File

@ -13,9 +13,9 @@ ZooKeeper is one of the first well-known open-source coordination systems. It's
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.
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 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.
:::note
:::note
External integrations are not supported.
:::
@ -117,7 +117,7 @@ clickhouse keeper --config /etc/your_path_to_config/config.xml
## Four Letter Word Commands {#four-letter-word-commands}
ClickHouse Keeper also provides 4lw commands which are almost the same with Zookeeper. Each command is composed of four letters such as `mntr`, `stat` etc. There are some more interesting commands: `stat` gives some general information about the server and connected clients, while `srvr` and `cons` give extended details on server and connections respectively.
ClickHouse Keeper also provides 4lw commands which are almost the same with Zookeeper. Each command is composed of four letters such as `mntr`, `stat` etc. There are some more interesting commands: `stat` gives some general information about the server and connected clients, while `srvr` and `cons` give extended details on server and connections respectively.
The 4lw commands has a white list configuration `four_letter_word_white_list` which has default value "conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro".
@ -319,3 +319,28 @@ clickhouse-keeper-converter --zookeeper-logs-dir /var/lib/zookeeper/version-2 --
4. Copy snapshot to ClickHouse server nodes with a configured `keeper` or start ClickHouse Keeper instead of ZooKeeper. The snapshot must persist on all nodes, otherwise, empty nodes can be faster and one of them can become a leader.
[Original article](https://clickhouse.com/docs/en/operations/clickhouse-keeper/) <!--hide-->
## Recovering after losing quorum
Because Clickhouse Keeper uses Raft it can tolerate certain amount of node crashes depending on the cluster size. \
E.g. for a 3-node cluster, it will continue working correctly if only 1 node crashes.
Cluster configuration can be dynamically configured but there are some limitations. Reconfiguration relies on Raft also
so to add/remove a node from the cluster you need to have a quorum. If you lose too many nodes in your cluster at the same time without any chance
of starting them again, Raft will stop working and not allow you to reconfigure your cluster using the convenvtional way.
Nevertheless, Clickhouse Keeper has a recovery mode which allows you to forcfully reconfigure your cluster with only 1 node.
This should be done only as your last resort if you cannot start your nodes again, or start a new instance on the same endpoint.
Important things to note before continuing:
- Make sure that the failed nodes cannot connect to the cluster again.
- Do not start any of the new nodes until it's specified in the steps.
After making sure that the above things are true, you need to do following:
1. Pick a single Keeper node to be your new leader. Be aware that the data of that node will be used for the entire cluster so we recommend to use a node with the most up to date state.
2. Before doing anything else, make a backup of the `log_storage_path` and `snapshot_storage_path` folders of the picked node.
3. Reconfigure the cluster on all of the nodes you want to use.
4. Send the four letter command `rcvr` to the node you picked which will move the node to the recovery mode OR stop Keeper instance on the picked node and start it again with the `--force-recovery` argument.
5. One by one, start Keeper instances on the new nodes making sure that `mntr` returns `follower` for the `zk_server_state` before starting the next one.
6. While in the recovery mode, the leader node will return error message for `mntr` command until it achieves quorum with the new nodes and refuse any requests from the client and the followers.
7. After quorum is achieved, the leader node will return to the normal mode of operation, accepting all the requests using Raft - verify with `mntr` which should return `leader` for the `zk_server_state`.

View File

@ -1 +0,0 @@
../../en/operations/clickhouse-keeper.md

View File

@ -0,0 +1,325 @@
---
sidebar_position: 66
sidebar_label: ClickHouse Keeper
---
# [预发生产] ClickHouse Keeper {#clickHouse-keeper}
ClickHouse 服务为了 [副本](../engines/table-engines/mergetree-family/replication.md) 和 [分布式DDL](../sql-reference/distributed-ddl.md) 查询执行使用 [ZooKeeper](https://zookeeper.apache.org/) 协调系统. ClickHouse Keeper 和 ZooKeeper是相互兼容的可互相替代.
:::danger "警告"
这个功能当前还在预发生产阶段. 我们只是在内部部分使用于生产环境和测试CI中.
:::
## 实现细节 {#implementation-details}
ZooKeeper最早的非常著名的开源协调系统之一. 它是通过Java语言实现的, 有一个相当节点和强大的数据模型. ZooKeeper的协调算法叫做 ZAB (ZooKeeper Atomic Broadcast) zk不能保证读取的线性化, 以为每个zk节点服务都是通过本地线性读的. ClickHouse Keeper是通过C++写的和zookeeper不一样 ClickHouse Keeper使用的[RAFT algorithm](https://raft.github.io/) [implementation](https://github.com/eBay/NuRaft)算法. 这个算法允许线性读和写, 已经有几种不同的语言的开源实现.
ClickHouse Keeper 默认提供了一些保证和ZooKeeper是一样的 (线性写, 非线性读)和. clickhouse keeper有一个兼容的客户端服务端协议, 所以任何标准的zookeeper客户端都可以用来与clickhouse keeper进行交互. 快照和日志的格式与ZooKeeper不兼容, 但是通过`clickhouse-keeper-converter` 允许转换 ZooKeeper 数据到 ClickHouse Keeper 快照. ClickHouse Keeper的interserver协议和zookeeper也不兼容所以ZooKeeper / ClickHouse Keeper 混合部署集群是不可能的.
ClickHouse Keeper支持访问控制列表(ACL)的方式和[ZooKeeper](https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html#sc_ZooKeeperAccessControl) 一样. ClickHouse Keeper支持相同的权限集合并且有完全相同的内置方案如`world`, `auth`, `digest`, `host` and `ip`. 权限校验使用 `用户名:密码`方式配对. 密码通过Base64算法加密.
:::info "注意"
不支持外部集成
:::
## 配置 {#configuration}
ClickHouse Keeper 完全可以作为ZooKeeper的独立替代品或者作为ClickHouse server服务的内部组件, 但是这两种方式下的配置使用 `.xml` 格式也几乎都是相同的. ClickHouse Keeper 配置的标签是 `<keeper_server>`. Keeper 配置有以下这些参数:
- `tcp_port` — 客户端连接的端口(ZooKeeper默认是`2181`).
- `tcp_port_secure` — client 和 keeper-server之间的SSL连接的安全端口.
- `server_id` — 唯一的服务器ID, ClickHouse Keeper 集群的每个参与组件都必须有一个唯一的编号(1, 2, 3, 等等).
- `log_storage_path` — 协调日志的路径, 最好存放在不繁忙的机器上 (和ZooKeeper一样).
- `snapshot_storage_path` — 协调快照的路径.
其他常见参数继承自ClickHouse server的配置 (`listen_host`, `logger`, 等等).
内部协调配置位于`<keeper_server>.<coordination_settings>`部分:
- `operation_timeout_ms` — 单个客户端操作的超时时间(ms)(默认值:10000)。
- `min_session_timeout_ms` — 客户端会话的最小超时时间(ms)(默认值:10000)。
- `session_timeout_ms` — 客户端会话最大超时时间(ms)(默认100000)。
- `dead_session_check_period_ms` — ClickHouse Keeper检查死会话并删除它们的频率(毫秒)(默认值:500)。
- `heart_beat_interval_ms` — ClickHouse Keeper的leader发送心跳频率(毫秒)(默认为500)。
- `election_timeout_lower_bound_ms` — 如果follower在此间隔内没有收到leader的心跳那么它可以启动leader选举(默认为1000).
- `election_timeout_upper_bound_ms` — 如果follower在此间隔内没有收到leader的心跳那么它必须启动leader选举(默认为2000)。
- `rotate_log_storage_interval` — 单个文件中存储的日志记录数量(默认100000条)。
- `reserved_log_items` — 在压缩之前需要存储多少协调日志记录(默认100000)。
- `snapshot_distance` — ClickHouse Keeper创建新快照的频率(以日志记录的数量为单位)(默认100000)。
- `snapshots_to_keep` — 保留多少个快照(默认值:3)。
- `stale_log_gap` — 当leader认为follower过时并发送快照给follower而不是日志时的阈值(默认值:10000)。
- `fresh_log_gap` — 当节点变成新鲜时的间隔(默认值:200)。
- `max_requests_batch_size` - 发送到RAFT之前的最大批量请求数(默认值:100)。
- `force_sync` — 在每次写入协调日志时是否调用' fsync '(默认值:true)。
- `quorum_reads` — 通过整个RAFT共识以类似的速度执行读请求和写请求(默认值:false)。
- `raft_logs_level` — 关于协调的文本日志级别 (trace, debug, 等等) (默认: system default).
- `auto_forwarding` — 允许将follower的请求转发到leader (默认: true).
- `shutdown_timeout` — 等待内部连接完成并关闭(ms)(默认值:5000)。
- `startup_timeout` — 如果服务器在指定的超时时间内没有连接到其他仲裁参与者,它将终止(ms)(默认值:30000)。
- `four_letter_word_white_list` — 4个字母的白名单列表 (默认: "conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro").
仲裁配置位于 `<keeper_server>.<raft_configuration>` 部分,并且保护一些描述
整个仲裁的唯一参数是“secure”它为仲裁参与者之间的通信启用加密连接。如果节点之间的内部通信需要SSL连接则该参数可以设置为“true”否则不指定。
每个`<server>`的主要参数是:
- `id` — 仲裁中的服务器标识符。
- `hostname` — 放置该服务器的主机名。
- `port` — 服务器监听连接的端口。
在[integration tests](https://github.com/ClickHouse/ClickHouse/tree/master/tests/integration)可以找到带有 `test_keeper_` 前缀的3个节点的仲裁配置示例. 服务配置举例如下 #1:
```xml
<keeper_server>
<tcp_port>2181</tcp_port>
<server_id>1</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
<coordination_settings>
<operation_timeout_ms>10000</operation_timeout_ms>
<session_timeout_ms>30000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>zoo1</hostname>
<port>9444</port>
</server>
<server>
<id>2</id>
<hostname>zoo2</hostname>
<port>9444</port>
</server>
<server>
<id>3</id>
<hostname>zoo3</hostname>
<port>9444</port>
</server>
</raft_configuration>
</keeper_server>
```
## 如何运行 {#how-to-run}
ClickHouse Keeper被绑定到ClickHouse服务器包中只需添加配置' <keeper_server> '并像往常一样启动ClickHouse服务器。如果你想运行独立的ClickHouse Keeper你可以用类似的方式启动它:
```bash
clickhouse-keeper --config /etc/your_path_to_config/config.xml
```
如果你没有符号链接(' clickhouse-keeper '),你可以创建它或指定' keeper '作为参数:
```bash
clickhouse keeper --config /etc/your_path_to_config/config.xml
```
## 四字母命令 {#four-letter-word-commands}
ClickHouse Keeper还提供了与Zookeeper几乎相同的4lw命令。每个命令由4个字母组成如“mntr”、“stat”等。还有一些更有趣的命令:' stat '给出了服务器和连接客户端的一般信息,而' srvr '和' cons '分别给出了服务器和连接的详细信息。
4lw命令有一个白名单配置“four_letter_word_white_list”它的默认值为“conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro”。
您可以通过telnet或nc在客户端端口向ClickHouse Keeper发出命令。
```
echo mntr | nc localhost 9181
```
下面是4lw的详细命令:
- `ruok`: 测试服务器运行时是否处于无错误状态。如果服务器正在运行它将用imok响应。否则它将完全不响应。“imok”的响应并不一定表明服务器已加入仲裁只是表明服务器进程处于活动状态并绑定到指定的客户端端口。使用“stat”获取状态wrt仲裁和客户端连接信息的详细信息。
```
imok
```
- `mntr`: 输出可用于监视集群运行状况的变量列表。
```
zk_version v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7
zk_avg_latency 0
zk_max_latency 0
zk_min_latency 0
zk_packets_received 68
zk_packets_sent 68
zk_num_alive_connections 1
zk_outstanding_requests 0
zk_server_state leader
zk_znode_count 4
zk_watch_count 1
zk_ephemerals_count 0
zk_approximate_data_size 723
zk_open_file_descriptor_count 310
zk_max_file_descriptor_count 10240
zk_followers 0
zk_synced_followers 0
```
- `srvr`: 列出服务器的完整详细信息。
```
ClickHouse Keeper version: v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7
Latency min/avg/max: 0/0/0
Received: 2
Sent : 2
Connections: 1
Outstanding: 0
Zxid: 34
Mode: leader
Node count: 4
```
- `stat`: 列出服务器和连接客户机的简要详细信息。
```
ClickHouse Keeper version: v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7
Clients:
192.168.1.1:52852(recved=0,sent=0)
192.168.1.1:52042(recved=24,sent=48)
Latency min/avg/max: 0/0/0
Received: 4
Sent : 4
Connections: 1
Outstanding: 0
Zxid: 36
Mode: leader
Node count: 4
```
- `srst`: 重置服务器统计数据。该命令将影响' srvr ' ' mntr '和' stat '的结果。
```
Server stats reset.
```
- `conf`: 打印服务配置详细信息。
```
server_id=1
tcp_port=2181
four_letter_word_white_list=*
log_storage_path=./coordination/logs
snapshot_storage_path=./coordination/snapshots
max_requests_batch_size=100
session_timeout_ms=30000
operation_timeout_ms=10000
dead_session_check_period_ms=500
heart_beat_interval_ms=500
election_timeout_lower_bound_ms=1000
election_timeout_upper_bound_ms=2000
reserved_log_items=1000000000000000
snapshot_distance=10000
auto_forwarding=true
shutdown_timeout=5000
startup_timeout=240000
raft_logs_level=information
snapshots_to_keep=3
rotate_log_storage_interval=100000
stale_log_gap=10000
fresh_log_gap=200
max_requests_batch_size=100
quorum_reads=false
force_sync=false
compress_logs=true
compress_snapshots_with_zstd_format=true
configuration_change_tries_count=20
```
- `cons`: 列出所有连接到此服务器的客户端的完整连接/会话详细信息。包括接收/发送的包数、会话id、操作延迟、最后执行的操作等信息。
```
192.168.1.1:52163(recved=0,sent=0,sid=0xffffffffffffffff,lop=NA,est=1636454787393,to=30000,lzxid=0xffffffffffffffff,lresp=0,llat=0,minlat=0,avglat=0,maxlat=0)
192.168.1.1:52042(recved=9,sent=18,sid=0x0000000000000001,lop=List,est=1636454739887,to=30000,lcxid=0x0000000000000005,lzxid=0x0000000000000005,lresp=1636454739892,llat=0,minlat=0,avglat=0,maxlat=0)
```
- `crst`: 重置所有连接的连接/会话统计信息。
```
Connection stats reset.
```
- `envi`: 打印服务环境详细信息
```
Environment:
clickhouse.keeper.version=v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7
host.name=ZBMAC-C02D4054M.local
os.name=Darwin
os.arch=x86_64
os.version=19.6.0
cpu.count=12
user.name=root
user.home=/Users/JackyWoo/
user.dir=/Users/JackyWoo/project/jd/clickhouse/cmake-build-debug/programs/
user.tmp=/var/folders/b4/smbq5mfj7578f2jzwn602tt40000gn/T/
```
- `dirs`: 以字节为单位显示快照和日志文件的总大小
```
snapshot_dir_size: 0
log_dir_size: 3875
```
- `isro`: 测试服务器是否以只读模式运行。如果处于只读模式服务器将响应“ro”如果不是只读模式则响应“rw”。
```
rw
```
- `wchs`: 列出服务器的监视的简要信息。
```
1 connections watching 1 paths
Total watches:1
```
- `wchc`: 按会话列出服务器的监视的详细信息。这将输出一个会话(连接)列表和相关的监视(路径)。注意,根据监视的数量,此操作可能会很昂贵(即影响服务器性能),请谨慎使用。
```
0x0000000000000001
/clickhouse/task_queue/ddl
```
- `wchp`: 按路径列出有关服务器的监视的详细信息。这将输出一个带有关联会话的路径(znode)列表。注意,根据监视的数量,此操作可能昂贵(即影响服务器性能),请谨慎使用。
```
/clickhouse/task_queue/ddl
0x0000000000000001
```
- `dump`: 列出未完成的会话和临时节点。这只对领导者有效。
```
Sessions dump (2):
0x0000000000000001
0x0000000000000002
Sessions with Ephemerals (1):
0x0000000000000001
/clickhouse/task_queue/ddl
```
## [实现] 从ZooKeeper迁移 {#migration-from-zookeeper}
从ZooKeeper无缝迁移到ClickHouse Keeper是不可能的你必须停止你的ZooKeeper集群转换数据并启动ClickHouse Keeper。' ClickHouse - Keeper -converter '工具允许将ZooKeeper日志和快照转换为ClickHouse Keeper快照。它只适用于ZooKeeper 大于 3.4。迁移的步骤:
1. 停掉ZooKeeper节点.
2. 可选,但建议:找到ZooKeeper leader节点重新启停。它会强制ZooKeeper创建一致的快照。
3. 在leader节点运行`clickhouse-keeper-converter`, 如下:
```bash
clickhouse-keeper-converter --zookeeper-logs-dir /var/lib/zookeeper/version-2 --zookeeper-snapshots-dir /var/lib/zookeeper/version-2 --output-dir /path/to/clickhouse/keeper/snapshots
```
4. 将快照复制到配置了“keeper”的ClickHouse服务器节点或者启动ClickHouse keeper而不是ZooKeeper。快照必须在所有节点上持久保存否则空节点可能更快其中一个节点可能成为leader.
[Original article](https://clickhouse.com/docs/en/operations/clickhouse-keeper/) <!--hide-->

View File

@ -548,11 +548,13 @@ private:
json_out << double_quote << connections[i]->getDescription() << ": {\n";
json_out << double_quote << "statistics" << ": {\n";
print_key_value("QPS", info->queries / info->work_time);
print_key_value("RPS", info->read_rows / info->work_time);
print_key_value("MiBPS", info->read_bytes / info->work_time);
print_key_value("RPS_result", info->result_rows / info->work_time);
print_key_value("MiBPS_result", info->result_bytes / info->work_time);
double seconds = info->work_time / concurrency;
print_key_value("QPS", info->queries.load() / seconds);
print_key_value("RPS", info->read_rows / seconds);
print_key_value("MiBPS", info->read_bytes / seconds / 1048576);
print_key_value("RPS_result", info->result_rows / seconds);
print_key_value("MiBPS_result", info->result_bytes / seconds / 1048576);
print_key_value("num_queries", info->queries.load());
print_key_value("num_errors", info->errors, false);

View File

@ -62,6 +62,7 @@ namespace ErrorCodes
extern const int NETWORK_ERROR;
extern const int MISMATCHING_USERS_FOR_PROCESS_AND_DATA;
extern const int FAILED_TO_GETPWUID;
extern const int LOGICAL_ERROR;
}
namespace
@ -239,6 +240,18 @@ std::string Keeper::getDefaultConfigFileName() const
return "keeper_config.xml";
}
void Keeper::handleCustomArguments(const std::string & arg, [[maybe_unused]] const std::string & value) // NOLINT
{
if (arg == "force-recovery")
{
assert(value.empty());
config().setBool("keeper_server.force_recovery", true);
return;
}
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid argument {} provided", arg);
}
void Keeper::defineOptions(Poco::Util::OptionSet & options)
{
options.addOption(
@ -251,6 +264,12 @@ void Keeper::defineOptions(Poco::Util::OptionSet & options)
.required(false)
.repeatable(false)
.binding("version"));
options.addOption(
Poco::Util::Option("force-recovery", "force-recovery", "Force recovery mode allowing Keeper to overwrite cluster configuration without quorum")
.required(false)
.repeatable(false)
.noArgument()
.callback(Poco::Util::OptionCallback<Keeper>(this, &Keeper::handleCustomArguments)));
BaseDaemon::defineOptions(options);
}

View File

@ -43,6 +43,8 @@ public:
protected:
void logRevision() const override;
void handleCustomArguments(const std::string & arg, const std::string & value);
int run() override;
void initialize(Application & self) override;

View File

@ -1,120 +0,0 @@
#include <Backups/ArchiveBackup.h>
#include <Disks/IDisk.h>
#include <IO/ReadBufferFromFileBase.h>
#include <IO/WriteBufferFromFileBase.h>
#include <IO/Archives/IArchiveReader.h>
#include <IO/Archives/IArchiveWriter.h>
#include <IO/Archives/createArchiveReader.h>
#include <IO/Archives/createArchiveWriter.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
ArchiveBackup::ArchiveBackup(
const String & backup_name_,
const DiskPtr & disk_,
const String & path_,
const ContextPtr & context_,
const std::optional<BackupInfo> & base_backup_info_)
: BackupImpl(backup_name_, context_, base_backup_info_), disk(disk_), path(path_)
{
}
ArchiveBackup::~ArchiveBackup()
{
close();
}
bool ArchiveBackup::backupExists() const
{
return disk ? disk->exists(path) : fs::exists(path);
}
void ArchiveBackup::openImpl(OpenMode open_mode_)
{
/// mutex is already locked
if (open_mode_ == OpenMode::WRITE)
{
/// Create a directory to contain the archive.
auto dir_path = fs::path(path).parent_path();
if (disk)
disk->createDirectories(dir_path);
else
std::filesystem::create_directories(dir_path);
/// Start writing the archive.
if (disk)
writer = createArchiveWriter(path, disk->writeFile(path));
else
writer = createArchiveWriter(path);
writer->setCompression(compression_method, compression_level);
writer->setPassword(password);
}
else if (open_mode_ == OpenMode::READ)
{
if (disk)
{
auto archive_read_function = [d = disk, p = path]() -> std::unique_ptr<SeekableReadBuffer> { return d->readFile(p); };
size_t archive_size = disk->getFileSize(path);
reader = createArchiveReader(path, archive_read_function, archive_size);
}
else
reader = createArchiveReader(path);
reader->setPassword(password);
}
}
void ArchiveBackup::closeImpl(const Strings &, bool writing_finalized_)
{
/// mutex is already locked
if (writer && writer->isWritingFile())
throw Exception("There is some writing unfinished on close", ErrorCodes::LOGICAL_ERROR);
writer.reset();
reader.reset();
if ((getOpenModeNoLock() == OpenMode::WRITE) && !writing_finalized_)
fs::remove(path);
}
std::unique_ptr<ReadBuffer> ArchiveBackup::readFileImpl(const String & file_name) const
{
/// mutex is already locked
return reader->readFile(file_name);
}
std::unique_ptr<WriteBuffer> ArchiveBackup::writeFileImpl(const String & file_name)
{
/// mutex is already locked
return writer->writeFile(file_name);
}
void ArchiveBackup::setCompression(const String & compression_method_, int compression_level_)
{
std::lock_guard lock{mutex};
compression_method = compression_method_;
compression_level = compression_level_;
if (writer)
writer->setCompression(compression_method, compression_level);
}
void ArchiveBackup::setPassword(const String & password_)
{
std::lock_guard lock{mutex};
password = password_;
if (writer)
writer->setPassword(password);
if (reader)
reader->setPassword(password);
}
}

View File

@ -1,52 +0,0 @@
#pragma once
#include <Backups/BackupImpl.h>
namespace DB
{
class IDisk;
using DiskPtr = std::shared_ptr<IDisk>;
class IArchiveReader;
class IArchiveWriter;
/// Stores a backup as a single .zip file.
class ArchiveBackup : public BackupImpl
{
public:
/// `disk`_ is allowed to be nullptr and that means the `path_` is a path in the local filesystem.
ArchiveBackup(
const String & backup_name_,
const DiskPtr & disk_,
const String & path_,
const ContextPtr & context_,
const std::optional<BackupInfo> & base_backup_info_ = {});
~ArchiveBackup() override;
static constexpr const int kDefaultCompressionLevel = -1;
/// Sets compression method and level.
void setCompression(const String & compression_method_, int compression_level_ = kDefaultCompressionLevel);
/// Sets password.
void setPassword(const String & password_);
private:
bool backupExists() const override;
void openImpl(OpenMode open_mode_) override;
void closeImpl(const Strings & written_files_, bool writing_finalized_) override;
bool supportsWritingInMultipleThreads() const override { return false; }
std::unique_ptr<ReadBuffer> readFileImpl(const String & file_name) const override;
std::unique_ptr<WriteBuffer> writeFileImpl(const String & file_name) override;
const DiskPtr disk;
const String path;
std::shared_ptr<IArchiveReader> reader;
std::shared_ptr<IArchiveWriter> writer;
String compression_method;
int compression_level = kDefaultCompressionLevel;
String password;
};
}

View File

@ -0,0 +1,262 @@
#include <Backups/BackupCoordinationDistributed.h>
#include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/escapeForFileName.h>
#include <Common/hex.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNEXPECTED_NODE_IN_ZOOKEEPER;
}
/// zookeeper_path/file_names/file_name->checksum_and_size
/// zookeeper_path/file_infos/checksum_and_size->info
/// zookeeper_path/archive_suffixes
/// zookeeper_path/current_archive_suffix
namespace
{
using SizeAndChecksum = IBackupCoordination::SizeAndChecksum;
using FileInfo = IBackupCoordination::FileInfo;
String serializeFileInfo(const FileInfo & info)
{
WriteBufferFromOwnString out;
writeBinary(info.file_name, out);
writeBinary(info.size, out);
writeBinary(info.checksum, out);
writeBinary(info.base_size, out);
writeBinary(info.base_checksum, out);
writeBinary(info.data_file_name, out);
writeBinary(info.archive_suffix, out);
writeBinary(info.pos_in_archive, out);
return out.str();
}
FileInfo deserializeFileInfo(const String & str)
{
FileInfo info;
ReadBufferFromString in{str};
readBinary(info.file_name, in);
readBinary(info.size, in);
readBinary(info.checksum, in);
readBinary(info.base_size, in);
readBinary(info.base_checksum, in);
readBinary(info.data_file_name, in);
readBinary(info.archive_suffix, in);
readBinary(info.pos_in_archive, in);
return info;
}
String serializeSizeAndChecksum(const SizeAndChecksum & size_and_checksum)
{
return getHexUIntLowercase(size_and_checksum.second) + '_' + std::to_string(size_and_checksum.first);
}
SizeAndChecksum deserializeSizeAndChecksum(const String & str)
{
constexpr size_t num_chars_in_checksum = sizeof(UInt128) * 2;
if (str.size() <= num_chars_in_checksum)
throw Exception(
ErrorCodes::UNEXPECTED_NODE_IN_ZOOKEEPER,
"Unexpected size of checksum: {}, must be {}",
str.size(),
num_chars_in_checksum);
UInt128 checksum = unhexUInt<UInt128>(str.data());
UInt64 size = parseFromString<UInt64>(str.substr(num_chars_in_checksum + 1));
return std::pair{size, checksum};
}
size_t extractCounterFromSequentialNodeName(const String & node_name)
{
size_t pos_before_counter = node_name.find_last_not_of("0123456789");
size_t counter_length = node_name.length() - 1 - pos_before_counter;
auto counter = std::string_view{node_name}.substr(node_name.length() - counter_length);
return parseFromString<UInt64>(counter);
}
String formatArchiveSuffix(size_t counter)
{
return fmt::format("{:03}", counter); /// Outputs 001, 002, 003, ...
}
/// We try to store data to zookeeper several times due to possible version conflicts.
constexpr size_t NUM_ATTEMPTS = 10;
}
BackupCoordinationDistributed::BackupCoordinationDistributed(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_)
: zookeeper_path(zookeeper_path_), get_zookeeper(get_zookeeper_)
{
createRootNodes();
}
BackupCoordinationDistributed::~BackupCoordinationDistributed() = default;
void BackupCoordinationDistributed::createRootNodes()
{
auto zookeeper = get_zookeeper();
zookeeper->createAncestors(zookeeper_path);
zookeeper->createIfNotExists(zookeeper_path, "");
zookeeper->createIfNotExists(zookeeper_path + "/file_names", "");
zookeeper->createIfNotExists(zookeeper_path + "/file_infos", "");
zookeeper->createIfNotExists(zookeeper_path + "/archive_suffixes", "");
}
void BackupCoordinationDistributed::removeAllNodes()
{
auto zookeeper = get_zookeeper();
zookeeper->removeRecursive(zookeeper_path);
}
void BackupCoordinationDistributed::addFileInfo(const FileInfo & file_info, bool & is_data_file_required)
{
auto zookeeper = get_zookeeper();
String full_path = zookeeper_path + "/file_names/" + escapeForFileName(file_info.file_name);
String size_and_checksum = serializeSizeAndChecksum(std::pair{file_info.size, file_info.checksum});
zookeeper->create(full_path, size_and_checksum, zkutil::CreateMode::Persistent);
if (!file_info.size)
{
is_data_file_required = false;
return;
}
full_path = zookeeper_path + "/file_infos/" + size_and_checksum;
auto code = zookeeper->tryCreate(full_path, serializeFileInfo(file_info), zkutil::CreateMode::Persistent);
if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS))
throw zkutil::KeeperException(code, full_path);
is_data_file_required = (code == Coordination::Error::ZOK) && (file_info.size > file_info.base_size);
}
void BackupCoordinationDistributed::updateFileInfo(const FileInfo & file_info)
{
if (!file_info.size)
return; /// we don't keep FileInfos for empty files, nothing to update
auto zookeeper = get_zookeeper();
String size_and_checksum = serializeSizeAndChecksum(std::pair{file_info.size, file_info.checksum});
String full_path = zookeeper_path + "/file_infos/" + size_and_checksum;
for (size_t attempt = 0; attempt < NUM_ATTEMPTS; ++attempt)
{
Coordination::Stat stat;
auto new_info = deserializeFileInfo(zookeeper->get(full_path, &stat));
new_info.archive_suffix = file_info.archive_suffix;
auto code = zookeeper->trySet(full_path, serializeFileInfo(new_info), stat.version);
if (code == Coordination::Error::ZOK)
return;
bool is_last_attempt = (attempt == NUM_ATTEMPTS - 1);
if ((code != Coordination::Error::ZBADVERSION) || is_last_attempt)
throw zkutil::KeeperException(code, full_path);
}
}
std::vector<FileInfo> BackupCoordinationDistributed::getAllFileInfos() const
{
auto zookeeper = get_zookeeper();
std::vector<FileInfo> file_infos;
Strings escaped_names = zookeeper->getChildren(zookeeper_path + "/file_names");
for (const String & escaped_name : escaped_names)
{
String size_and_checksum = zookeeper->get(zookeeper_path + "/file_names/" + escaped_name);
UInt64 size = deserializeSizeAndChecksum(size_and_checksum).first;
FileInfo file_info;
if (size) /// we don't keep FileInfos for empty files
file_info = deserializeFileInfo(zookeeper->get(zookeeper_path + "/file_infos/" + size_and_checksum));
file_info.file_name = unescapeForFileName(escaped_name);
file_infos.emplace_back(std::move(file_info));
}
return file_infos;
}
Strings BackupCoordinationDistributed::listFiles(const String & prefix, const String & terminator) const
{
auto zookeeper = get_zookeeper();
Strings escaped_names = zookeeper->getChildren(zookeeper_path + "/file_names");
Strings elements;
for (const String & escaped_name : escaped_names)
{
String name = unescapeForFileName(escaped_name);
if (!name.starts_with(prefix))
continue;
size_t start_pos = prefix.length();
size_t end_pos = String::npos;
if (!terminator.empty())
end_pos = name.find(terminator, start_pos);
std::string_view new_element = std::string_view{name}.substr(start_pos, end_pos - start_pos);
if (!elements.empty() && (elements.back() == new_element))
continue;
elements.push_back(String{new_element});
}
std::sort(elements.begin(), elements.end());
return elements;
}
std::optional<FileInfo> BackupCoordinationDistributed::getFileInfo(const String & file_name) const
{
auto zookeeper = get_zookeeper();
String size_and_checksum;
if (!zookeeper->tryGet(zookeeper_path + "/file_names/" + escapeForFileName(file_name), size_and_checksum))
return std::nullopt;
UInt64 size = deserializeSizeAndChecksum(size_and_checksum).first;
FileInfo file_info;
if (size) /// we don't keep FileInfos for empty files
file_info = deserializeFileInfo(zookeeper->get(zookeeper_path + "/file_infos/" + size_and_checksum));
file_info.file_name = file_name;
return file_info;
}
std::optional<FileInfo> BackupCoordinationDistributed::getFileInfo(const SizeAndChecksum & size_and_checksum) const
{
auto zookeeper = get_zookeeper();
String file_info_str;
if (!zookeeper->tryGet(zookeeper_path + "/file_infos/" + serializeSizeAndChecksum(size_and_checksum), file_info_str))
return std::nullopt;
return deserializeFileInfo(file_info_str);
}
std::optional<SizeAndChecksum> BackupCoordinationDistributed::getFileSizeAndChecksum(const String & file_name) const
{
auto zookeeper = get_zookeeper();
String size_and_checksum;
if (!zookeeper->tryGet(zookeeper_path + "/file_names/" + escapeForFileName(file_name), size_and_checksum))
return std::nullopt;
return deserializeSizeAndChecksum(size_and_checksum);
}
String BackupCoordinationDistributed::getNextArchiveSuffix()
{
auto zookeeper = get_zookeeper();
String path = zookeeper_path + "/archive_suffixes/a";
String path_created;
auto code = zookeeper->tryCreate(path, "", zkutil::CreateMode::PersistentSequential, path_created);
if (code != Coordination::Error::ZOK)
throw zkutil::KeeperException(code, path);
return formatArchiveSuffix(extractCounterFromSequentialNodeName(path_created));
}
Strings BackupCoordinationDistributed::getAllArchiveSuffixes() const
{
auto zookeeper = get_zookeeper();
Strings node_names = zookeeper->getChildren(zookeeper_path + "/archive_suffixes");
for (auto & node_name : node_names)
node_name = formatArchiveSuffix(extractCounterFromSequentialNodeName(node_name));
return node_names;
}
void BackupCoordinationDistributed::drop()
{
removeAllNodes();
}
}

View File

@ -0,0 +1,41 @@
#pragma once
#include <Backups/IBackupCoordination.h>
#include <Common/ZooKeeper/Common.h>
#include <map>
#include <unordered_map>
namespace DB
{
/// Stores backup temporary information in Zookeeper, used to perform BACKUP ON CLUSTER.
class BackupCoordinationDistributed : public IBackupCoordination
{
public:
BackupCoordinationDistributed(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_);
~BackupCoordinationDistributed() override;
void addFileInfo(const FileInfo & file_info, bool & is_data_file_required) override;
void updateFileInfo(const FileInfo & file_info) override;
std::vector<FileInfo> getAllFileInfos() const override;
Strings listFiles(const String & prefix, const String & terminator) const override;
std::optional<FileInfo> getFileInfo(const String & file_name) const override;
std::optional<FileInfo> getFileInfo(const SizeAndChecksum & size_and_checksum) const override;
std::optional<SizeAndChecksum> getFileSizeAndChecksum(const String & file_name) const override;
String getNextArchiveSuffix() override;
Strings getAllArchiveSuffixes() const override;
void drop() override;
private:
void createRootNodes();
void removeAllNodes();
const String zookeeper_path;
const zkutil::GetZooKeeper get_zookeeper;
};
}

View File

@ -0,0 +1,120 @@
#include <Backups/BackupCoordinationLocal.h>
#include <fmt/format.h>
namespace DB
{
using SizeAndChecksum = IBackupCoordination::SizeAndChecksum;
using FileInfo = IBackupCoordination::FileInfo;
BackupCoordinationLocal::BackupCoordinationLocal() = default;
BackupCoordinationLocal::~BackupCoordinationLocal() = default;
void BackupCoordinationLocal::addFileInfo(const FileInfo & file_info, bool & is_data_file_required)
{
std::lock_guard lock{mutex};
file_names.emplace(file_info.file_name, std::pair{file_info.size, file_info.checksum});
if (!file_info.size)
{
is_data_file_required = false;
return;
}
bool inserted_file_info = file_infos.try_emplace(std::pair{file_info.size, file_info.checksum}, file_info).second;
is_data_file_required = inserted_file_info && (file_info.size > file_info.base_size);
}
void BackupCoordinationLocal::updateFileInfo(const FileInfo & file_info)
{
if (!file_info.size)
return; /// we don't keep FileInfos for empty files, nothing to update
std::lock_guard lock{mutex};
auto & dest = file_infos.at(std::pair{file_info.size, file_info.checksum});
dest.archive_suffix = file_info.archive_suffix;
}
std::vector<FileInfo> BackupCoordinationLocal::getAllFileInfos() const
{
std::lock_guard lock{mutex};
std::vector<FileInfo> res;
for (const auto & [file_name, size_and_checksum] : file_names)
{
FileInfo info;
UInt64 size = size_and_checksum.first;
if (size) /// we don't keep FileInfos for empty files
info = file_infos.at(size_and_checksum);
info.file_name = file_name;
res.push_back(std::move(info));
}
return res;
}
Strings BackupCoordinationLocal::listFiles(const String & prefix, const String & terminator) const
{
std::lock_guard lock{mutex};
Strings elements;
for (auto it = file_names.lower_bound(prefix); it != file_names.end(); ++it)
{
const String & name = it->first;
if (!name.starts_with(prefix))
break;
size_t start_pos = prefix.length();
size_t end_pos = String::npos;
if (!terminator.empty())
end_pos = name.find(terminator, start_pos);
std::string_view new_element = std::string_view{name}.substr(start_pos, end_pos - start_pos);
if (!elements.empty() && (elements.back() == new_element))
continue;
elements.push_back(String{new_element});
}
return elements;
}
std::optional<FileInfo> BackupCoordinationLocal::getFileInfo(const String & file_name) const
{
std::lock_guard lock{mutex};
auto it = file_names.find(file_name);
if (it == file_names.end())
return std::nullopt;
const auto & size_and_checksum = it->second;
UInt64 size = size_and_checksum.first;
FileInfo info;
if (size) /// we don't keep FileInfos for empty files
info = file_infos.at(size_and_checksum);
info.file_name = file_name;
return info;
}
std::optional<FileInfo> BackupCoordinationLocal::getFileInfo(const SizeAndChecksum & size_and_checksum) const
{
std::lock_guard lock{mutex};
auto it = file_infos.find(size_and_checksum);
if (it == file_infos.end())
return std::nullopt;
return it->second;
}
std::optional<SizeAndChecksum> BackupCoordinationLocal::getFileSizeAndChecksum(const String & file_name) const
{
std::lock_guard lock{mutex};
auto it = file_names.find(file_name);
if (it == file_names.end())
return std::nullopt;
return it->second;
}
String BackupCoordinationLocal::getNextArchiveSuffix()
{
std::lock_guard lock{mutex};
String new_archive_suffix = fmt::format("{:03}", ++current_archive_suffix); /// Outputs 001, 002, 003, ...
archive_suffixes.push_back(new_archive_suffix);
return new_archive_suffix;
}
Strings BackupCoordinationLocal::getAllArchiveSuffixes() const
{
std::lock_guard lock{mutex};
return archive_suffixes;
}
}

View File

@ -0,0 +1,39 @@
#pragma once
#include <Backups/IBackupCoordination.h>
#include <map>
#include <mutex>
namespace DB
{
/// Stores backup contents information in memory.
class BackupCoordinationLocal : public IBackupCoordination
{
public:
BackupCoordinationLocal();
~BackupCoordinationLocal() override;
void addFileInfo(const FileInfo & file_info, bool & is_data_file_required) override;
void updateFileInfo(const FileInfo & file_info) override;
std::vector<FileInfo> getAllFileInfos() const override;
Strings listFiles(const String & prefix, const String & terminator) const override;
std::optional<FileInfo> getFileInfo(const String & file_name) const override;
std::optional<FileInfo> getFileInfo(const SizeAndChecksum & size_and_checksum) const override;
std::optional<SizeAndChecksum> getFileSizeAndChecksum(const String & file_name) const override;
String getNextArchiveSuffix() override;
Strings getAllArchiveSuffixes() const override;
private:
mutable std::mutex mutex;
std::map<String /* file_name */, SizeAndChecksum> file_names; /// Should be ordered alphabetically, see listFiles(). For empty files we assume checksum = 0.
std::map<SizeAndChecksum, FileInfo> file_infos; /// Information about files. Without empty files.
Strings archive_suffixes;
size_t current_archive_suffix = 0;
};
}

View File

@ -1,5 +1,5 @@
#include <Backups/BackupEntryFromAppendOnlyFile.h>
#include <IO/LimitReadBuffer.h>
#include <IO/LimitSeekableReadBuffer.h>
namespace DB
@ -26,10 +26,10 @@ BackupEntryFromAppendOnlyFile::BackupEntryFromAppendOnlyFile(
{
}
std::unique_ptr<ReadBuffer> BackupEntryFromAppendOnlyFile::getReadBuffer() const
std::unique_ptr<SeekableReadBuffer> BackupEntryFromAppendOnlyFile::getReadBuffer() const
{
auto buf = BackupEntryFromImmutableFile::getReadBuffer();
return std::make_unique<LimitReadBuffer>(std::move(buf), limit, false);
return std::make_unique<LimitSeekableReadBuffer>(std::move(buf), limit);
}
}

View File

@ -26,7 +26,7 @@ public:
const std::shared_ptr<TemporaryFileOnDisk> & temporary_file_ = {});
UInt64 getSize() const override { return limit; }
std::unique_ptr<ReadBuffer> getReadBuffer() const override;
std::unique_ptr<SeekableReadBuffer> getReadBuffer() const override;
private:
const UInt64 limit;

View File

@ -36,7 +36,7 @@ UInt64 BackupEntryFromImmutableFile::getSize() const
return *file_size;
}
std::unique_ptr<ReadBuffer> BackupEntryFromImmutableFile::getReadBuffer() const
std::unique_ptr<SeekableReadBuffer> BackupEntryFromImmutableFile::getReadBuffer() const
{
if (disk)
return disk->readFile(file_path);

View File

@ -33,7 +33,7 @@ public:
UInt64 getSize() const override;
std::optional<UInt128> getChecksum() const override { return checksum; }
std::unique_ptr<ReadBuffer> getReadBuffer() const override;
std::unique_ptr<SeekableReadBuffer> getReadBuffer() const override;
String getFilePath() const { return file_path; }
DiskPtr getDisk() const { return disk; }

View File

@ -15,7 +15,7 @@ BackupEntryFromMemory::BackupEntryFromMemory(String data_, const std::optional<U
{
}
std::unique_ptr<ReadBuffer> BackupEntryFromMemory::getReadBuffer() const
std::unique_ptr<SeekableReadBuffer> BackupEntryFromMemory::getReadBuffer() const
{
return std::make_unique<ReadBufferFromString>(data);
}

View File

@ -17,7 +17,7 @@ public:
UInt64 getSize() const override { return data.size(); }
std::optional<UInt128> getChecksum() const override { return checksum; }
std::unique_ptr<ReadBuffer> getReadBuffer() const override;
std::unique_ptr<SeekableReadBuffer> getReadBuffer() const override;
private:
const String data;

View File

@ -21,9 +21,7 @@ BackupMutablePtr BackupFactory::createBackup(const CreateParams & params) const
auto it = creators.find(engine_name);
if (it == creators.end())
throw Exception(ErrorCodes::BACKUP_ENGINE_NOT_FOUND, "Not found backup engine {}", engine_name);
BackupMutablePtr backup = (it->second)(params);
backup->open(params.open_mode);
return backup;
return (it->second)(params);
}
void BackupFactory::registerBackupEngine(const String & engine_name, const CreatorFn & creator_fn)

View File

@ -24,12 +24,15 @@ public:
struct CreateParams
{
OpenMode open_mode = OpenMode::WRITE;
std::optional<UUID> backup_uuid;
BackupInfo backup_info;
std::optional<BackupInfo> base_backup_info;
String compression_method;
int compression_level = -1;
String password;
ContextPtr context;
bool is_internal_backup = false;
String coordination_zk_path;
};
static BackupFactory & instance();

30
src/Backups/BackupIO.h Normal file
View File

@ -0,0 +1,30 @@
#pragma once
#include <Core/Types.h>
namespace DB
{
class SeekableReadBuffer;
class WriteBuffer;
/// Represents operations of loading from disk or downloading for reading a backup.
class IBackupReader /// BackupReaderFile, BackupReaderDisk, BackupReaderS3
{
public:
virtual ~IBackupReader() = default;
virtual bool fileExists(const String & file_name) = 0;
virtual size_t getFileSize(const String & file_name) = 0;
virtual std::unique_ptr<SeekableReadBuffer> readFile(const String & file_name) = 0;
};
/// Represents operations of storing to disk or uploading for writing a backup.
class IBackupWriter /// BackupWriterFile, BackupWriterDisk, BackupWriterS3
{
public:
virtual ~IBackupWriter() = default;
virtual bool fileExists(const String & file_name) = 0;
virtual std::unique_ptr<WriteBuffer> writeFile(const String & file_name) = 0;
virtual void removeFilesAfterFailure(const Strings & file_names) = 0;
};
}

View File

@ -0,0 +1,65 @@
#include <Backups/BackupIO_Disk.h>
#include <Common/Exception.h>
#include <Disks/IDisk.h>
#include <IO/ReadBufferFromFileBase.h>
#include <IO/WriteBufferFromFileBase.h>
#include <base/logger_useful.h>
namespace DB
{
BackupReaderDisk::BackupReaderDisk(const DiskPtr & disk_, const String & path_) : disk(disk_), path(path_)
{
}
BackupReaderDisk::~BackupReaderDisk() = default;
bool BackupReaderDisk::fileExists(const String & file_name)
{
return disk->exists(path / file_name);
}
size_t BackupReaderDisk::getFileSize(const String & file_name)
{
return disk->getFileSize(path / file_name);
}
std::unique_ptr<SeekableReadBuffer> BackupReaderDisk::readFile(const String & file_name)
{
return disk->readFile(path / file_name);
}
BackupWriterDisk::BackupWriterDisk(const DiskPtr & disk_, const String & path_) : disk(disk_), path(path_)
{
}
BackupWriterDisk::~BackupWriterDisk() = default;
bool BackupWriterDisk::fileExists(const String & file_name)
{
return disk->exists(path / file_name);
}
std::unique_ptr<WriteBuffer> BackupWriterDisk::writeFile(const String & file_name)
{
auto file_path = path / file_name;
disk->createDirectories(file_path.parent_path());
return disk->writeFile(file_path);
}
void BackupWriterDisk::removeFilesAfterFailure(const Strings & file_names)
{
try
{
for (const auto & file_name : file_names)
disk->removeFileIfExists(path / file_name);
if (disk->isDirectory(path) && disk->isDirectoryEmpty(path))
disk->removeDirectory(path);
}
catch (...)
{
LOG_WARNING(&Poco::Logger::get("BackupWriterDisk"), "RemoveFilesAfterFailure: {}", getCurrentExceptionMessage(false));
}
}
}

View File

@ -0,0 +1,41 @@
#pragma once
#include <filesystem>
#include <Backups/BackupIO.h>
namespace DB
{
class IDisk;
using DiskPtr = std::shared_ptr<IDisk>;
class BackupReaderDisk : public IBackupReader
{
public:
BackupReaderDisk(const DiskPtr & disk_, const String & path_);
~BackupReaderDisk() override;
bool fileExists(const String & file_name) override;
size_t getFileSize(const String & file_name) override;
std::unique_ptr<SeekableReadBuffer> readFile(const String & file_name) override;
private:
DiskPtr disk;
std::filesystem::path path;
};
class BackupWriterDisk : public IBackupWriter
{
public:
BackupWriterDisk(const DiskPtr & disk_, const String & path_);
~BackupWriterDisk() override;
bool fileExists(const String & file_name) override;
std::unique_ptr<WriteBuffer> writeFile(const String & file_name) override;
void removeFilesAfterFailure(const Strings & file_names) override;
private:
DiskPtr disk;
std::filesystem::path path;
};
}

View File

@ -0,0 +1,66 @@
#include <Backups/BackupIO_File.h>
#include <Common/Exception.h>
#include <Disks/IO/createReadBufferFromFileBase.h>
#include <IO/WriteBufferFromFile.h>
#include <base/logger_useful.h>
namespace fs = std::filesystem;
namespace DB
{
BackupReaderFile::BackupReaderFile(const String & path_) : path(path_)
{
}
BackupReaderFile::~BackupReaderFile() = default;
bool BackupReaderFile::fileExists(const String & file_name)
{
return fs::exists(path / file_name);
}
size_t BackupReaderFile::getFileSize(const String & file_name)
{
return fs::file_size(path / file_name);
}
std::unique_ptr<SeekableReadBuffer> BackupReaderFile::readFile(const String & file_name)
{
return createReadBufferFromFileBase(path / file_name, {});
}
BackupWriterFile::BackupWriterFile(const String & path_) : path(path_)
{
}
BackupWriterFile::~BackupWriterFile() = default;
bool BackupWriterFile::fileExists(const String & file_name)
{
return fs::exists(path / file_name);
}
std::unique_ptr<WriteBuffer> BackupWriterFile::writeFile(const String & file_name)
{
auto file_path = path / file_name;
fs::create_directories(file_path.parent_path());
return std::make_unique<WriteBufferFromFile>(file_path);
}
void BackupWriterFile::removeFilesAfterFailure(const Strings & file_names)
{
try
{
for (const auto & file_name : file_names)
fs::remove(path / file_name);
if (fs::is_directory(path) && fs::is_empty(path))
fs::remove(path);
}
catch (...)
{
LOG_WARNING(&Poco::Logger::get("BackupWriterFile"), "RemoveFilesAfterFailure: {}", getCurrentExceptionMessage(false));
}
}
}

View File

@ -0,0 +1,37 @@
#pragma once
#include <filesystem>
#include <Backups/BackupIO.h>
namespace DB
{
class BackupReaderFile : public IBackupReader
{
public:
BackupReaderFile(const String & path_);
~BackupReaderFile() override;
bool fileExists(const String & file_name) override;
size_t getFileSize(const String & file_name) override;
std::unique_ptr<SeekableReadBuffer> readFile(const String & file_name) override;
private:
std::filesystem::path path;
};
class BackupWriterFile : public IBackupWriter
{
public:
BackupWriterFile(const String & path_);
~BackupWriterFile() override;
bool fileExists(const String & file_name) override;
std::unique_ptr<WriteBuffer> writeFile(const String & file_name) override;
void removeFilesAfterFailure(const Strings & file_names) override;
private:
std::filesystem::path path;
};
}

View File

@ -1,11 +1,19 @@
#include <Backups/BackupImpl.h>
#include <Backups/BackupFactory.h>
#include <Backups/BackupEntryFromMemory.h>
#include <Backups/BackupIO.h>
#include <Backups/IBackupEntry.h>
#include <Backups/BackupCoordinationLocal.h>
#include <Backups/BackupCoordinationDistributed.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/hex.h>
#include <Common/quoteString.h>
#include <IO/ConcatReadBuffer.h>
#include <Interpreters/Context.h>
#include <IO/Archives/IArchiveReader.h>
#include <IO/Archives/IArchiveWriter.h>
#include <IO/Archives/createArchiveReader.h>
#include <IO/Archives/createArchiveWriter.h>
#include <IO/ConcatSeekableReadBuffer.h>
#include <IO/HashingReadBuffer.h>
#include <IO/ReadBufferFromFileBase.h>
#include <IO/ReadHelpers.h>
@ -14,7 +22,6 @@
#include <IO/WriteHelpers.h>
#include <IO/copyData.h>
#include <Poco/Util/XMLConfiguration.h>
#include <boost/range/adaptor/map.hpp>
namespace DB
@ -35,14 +42,29 @@ namespace ErrorCodes
namespace
{
const UInt64 BACKUP_VERSION = 1;
const UInt64 INITIAL_BACKUP_VERSION = 1;
const UInt64 CURRENT_BACKUP_VERSION = 1;
using SizeAndChecksum = IBackup::SizeAndChecksum;
using FileInfo = IBackupCoordination::FileInfo;
String hexChecksum(UInt128 checksum)
{
return getHexUIntLowercase(checksum);
}
UInt128 unhexChecksum(const String & checksum)
{
if (checksum.size() != sizeof(UInt128) * 2)
throw Exception(ErrorCodes::BACKUP_DAMAGED, "Unexpected size of checksum: {}, must be {}", checksum.size(), sizeof(UInt128) * 2);
constexpr size_t num_chars_in_checksum = sizeof(UInt128) * 2;
if (checksum.size() != num_chars_in_checksum)
throw Exception(ErrorCodes::BACKUP_DAMAGED, "Unexpected size of checksum: {}, must be {}", checksum.size(), num_chars_in_checksum);
return unhexUInt<UInt128>(checksum.data());
}
String formatSizeAndChecksum(const SizeAndChecksum & size_and_checksum)
{
return hexChecksum(size_and_checksum.second) + std::to_string(size_and_checksum.first);
}
}
@ -51,22 +73,28 @@ class BackupImpl::BackupEntryFromBackupImpl : public IBackupEntry
public:
BackupEntryFromBackupImpl(
const std::shared_ptr<const BackupImpl> & backup_,
const String & file_name_,
const String & archive_suffix_,
const String & data_file_name_,
UInt64 size_,
const std::optional<UInt128> checksum_,
const UInt128 checksum_,
BackupEntryPtr base_backup_entry_ = {})
: backup(backup_), file_name(file_name_), size(size_), checksum(checksum_),
: backup(backup_), archive_suffix(archive_suffix_), data_file_name(data_file_name_), size(size_), checksum(checksum_),
base_backup_entry(std::move(base_backup_entry_))
{
}
std::unique_ptr<ReadBuffer> getReadBuffer() const override
std::unique_ptr<SeekableReadBuffer> getReadBuffer() const override
{
auto read_buffer = backup->readFileImpl(file_name);
std::unique_ptr<SeekableReadBuffer> read_buffer;
if (backup->use_archives)
read_buffer = backup->getArchiveReader(archive_suffix)->readFile(data_file_name);
else
read_buffer = backup->reader->readFile(data_file_name);
if (base_backup_entry)
{
auto base_backup_read_buffer = base_backup_entry->getReadBuffer();
read_buffer = std::make_unique<ConcatReadBuffer>(std::move(base_backup_read_buffer), std::move(read_buffer));
size_t base_size = base_backup_entry->getSize();
read_buffer = std::make_unique<ConcatSeekableReadBuffer>(
base_backup_entry->getReadBuffer(), base_size, std::move(read_buffer), size - base_size);
}
return read_buffer;
}
@ -76,52 +104,105 @@ public:
private:
const std::shared_ptr<const BackupImpl> backup;
const String file_name;
const String archive_suffix;
const String data_file_name;
const UInt64 size;
const std::optional<UInt128> checksum;
const UInt128 checksum;
BackupEntryPtr base_backup_entry;
};
BackupImpl::BackupImpl(const String & backup_name_, const ContextPtr & context_, const std::optional<BackupInfo> & base_backup_info_)
: backup_name(backup_name_), context(context_), base_backup_info_param(base_backup_info_)
BackupImpl::BackupImpl(
const String & backup_name_,
const ArchiveParams & archive_params_,
const std::optional<BackupInfo> & base_backup_info_,
std::shared_ptr<IBackupReader> reader_,
const ContextPtr & context_)
: backup_name(backup_name_)
, archive_params(archive_params_)
, use_archives(!archive_params.archive_name.empty())
, open_mode(OpenMode::READ)
, reader(std::move(reader_))
, is_internal_backup(false)
, coordination(std::make_shared<BackupCoordinationLocal>())
, context(context_)
, version(INITIAL_BACKUP_VERSION)
, base_backup_info(base_backup_info_)
{
open();
}
BackupImpl::~BackupImpl() = default;
void BackupImpl::open(OpenMode open_mode_)
BackupImpl::BackupImpl(
const String & backup_name_,
const ArchiveParams & archive_params_,
const std::optional<BackupInfo> & base_backup_info_,
std::shared_ptr<IBackupWriter> writer_,
const ContextPtr & context_,
const std::optional<UUID> & backup_uuid_,
bool is_internal_backup_,
const String & coordination_zk_path_)
: backup_name(backup_name_)
, archive_params(archive_params_)
, use_archives(!archive_params.archive_name.empty())
, open_mode(OpenMode::WRITE)
, writer(std::move(writer_))
, is_internal_backup(is_internal_backup_)
, context(context_)
, uuid(backup_uuid_)
, version(CURRENT_BACKUP_VERSION)
, base_backup_info(base_backup_info_)
{
if (coordination_zk_path_.empty())
coordination = std::make_shared<BackupCoordinationLocal>();
else
coordination = std::make_shared<BackupCoordinationDistributed>(coordination_zk_path_, [&] { return context->getZooKeeper(); });
open();
}
BackupImpl::~BackupImpl()
{
close();
}
void BackupImpl::open()
{
std::lock_guard lock{mutex};
if (open_mode == open_mode_)
return;
if (open_mode != OpenMode::NONE)
throw Exception("Backup is already opened", ErrorCodes::LOGICAL_ERROR);
String file_name_to_check_existence;
if (use_archives)
file_name_to_check_existence = archive_params.archive_name;
else
file_name_to_check_existence = ".backup";
bool backup_exists = (open_mode == OpenMode::WRITE) ? writer->fileExists(file_name_to_check_existence) : reader->fileExists(file_name_to_check_existence);
if (open_mode_ == OpenMode::WRITE)
if (open_mode == OpenMode::WRITE)
{
if (backupExists())
throw Exception(ErrorCodes::BACKUP_ALREADY_EXISTS, "Backup {} already exists", getName());
if (backup_exists)
throw Exception(ErrorCodes::BACKUP_ALREADY_EXISTS, "Backup {} already exists", backup_name);
}
else
{
if (!backup_exists)
throw Exception(ErrorCodes::BACKUP_NOT_FOUND, "Backup {} not found", backup_name);
}
if (open_mode == OpenMode::WRITE)
{
timestamp = std::time(nullptr);
uuid = UUIDHelpers::generateV4();
if (!uuid)
uuid = UUIDHelpers::generateV4();
writing_finalized = false;
written_files.clear();
}
if (open_mode_ == OpenMode::READ)
{
if (!backupExists())
throw Exception(ErrorCodes::BACKUP_NOT_FOUND, "Backup {} not found", getName());
}
openImpl(open_mode_);
base_backup_info = base_backup_info_param;
if (open_mode_ == OpenMode::READ)
if (open_mode == OpenMode::READ)
readBackupMetadata();
assert(uuid); /// Backup's UUID must be loaded or generated at this point.
if (base_backup_info)
{
BackupFactory::CreateParams params;
@ -130,37 +211,30 @@ void BackupImpl::open(OpenMode open_mode_)
params.context = context;
base_backup = BackupFactory::instance().createBackup(params);
if (open_mode_ == OpenMode::WRITE)
if (open_mode == OpenMode::WRITE)
base_backup_uuid = base_backup->getUUID();
else if (base_backup_uuid != base_backup->getUUID())
throw Exception(ErrorCodes::WRONG_BASE_BACKUP, "Backup {}: The base backup {} has different UUID ({} != {})",
getName(), base_backup->getName(), toString(base_backup->getUUID()), (base_backup_uuid ? toString(*base_backup_uuid) : ""));
backup_name, base_backup->getName(), toString(base_backup->getUUID()), (base_backup_uuid ? toString(*base_backup_uuid) : ""));
}
open_mode = open_mode_;
}
void BackupImpl::close()
{
std::lock_guard lock{mutex};
if (open_mode == OpenMode::NONE)
return;
closeImpl(written_files, writing_finalized);
if (!is_internal_backup && writing_finalized)
writeBackupMetadata();
uuid = UUIDHelpers::Nil;
timestamp = 0;
base_backup_info.reset();
base_backup.reset();
base_backup_uuid.reset();
file_infos.clear();
open_mode = OpenMode::NONE;
}
archive_readers.clear();
for (auto & archive_writer : archive_writers)
archive_writer = {"", nullptr};
IBackup::OpenMode BackupImpl::getOpenMode() const
{
std::lock_guard lock{mutex};
return open_mode;
if (!is_internal_backup && writer && !writing_finalized)
removeAllFilesAfterFailure();
if (!is_internal_backup)
coordination->drop();
}
time_t BackupImpl::getTimestamp() const
@ -172,14 +246,14 @@ time_t BackupImpl::getTimestamp() const
void BackupImpl::writeBackupMetadata()
{
Poco::AutoPtr<Poco::Util::XMLConfiguration> config{new Poco::Util::XMLConfiguration()};
config->setUInt("version", BACKUP_VERSION);
config->setUInt("version", CURRENT_BACKUP_VERSION);
config->setString("timestamp", toString(LocalDateTime{timestamp}));
config->setString("uuid", toString(uuid));
config->setString("uuid", toString(*uuid));
if (base_backup_info)
{
bool base_backup_in_use = false;
for (const auto & [name, info] : file_infos)
for (const auto & info : coordination->getAllFileInfos())
{
if (info.base_size)
base_backup_in_use = true;
@ -193,23 +267,29 @@ void BackupImpl::writeBackupMetadata()
}
size_t index = 0;
for (const auto & [name, info] : file_infos)
for (const auto & info : coordination->getAllFileInfos())
{
String prefix = index ? "contents.file[" + std::to_string(index) + "]." : "contents.file.";
config->setString(prefix + "name", name);
config->setUInt(prefix + "size", info.size);
if (info.size)
{
config->setString(prefix + "checksum", getHexUIntLowercase(info.checksum));
config->setString(prefix + "name", info.file_name);
config->setString(prefix + "checksum", hexChecksum(info.checksum));
if (info.base_size)
{
config->setBool(prefix + "use_base", true);
if (info.base_size != info.size)
{
config->setUInt(prefix + "base_size", info.base_size);
config->setString(prefix + "base_checksum", getHexUIntLowercase(info.base_checksum));
config->setString(prefix + "base_checksum", hexChecksum(info.base_checksum));
}
}
if (!info.data_file_name.empty() && (info.data_file_name != info.file_name))
config->setString(prefix + "data_file", info.data_file_name);
if (!info.archive_suffix.empty())
config->setString(prefix + "archive_suffix", info.archive_suffix);
if (info.pos_in_archive != static_cast<size_t>(-1))
config->setUInt64(prefix + "pos_in_archive", info.pos_in_archive);
}
++index;
}
@ -217,23 +297,32 @@ void BackupImpl::writeBackupMetadata()
std::ostringstream stream; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
config->save(stream);
String str = stream.str();
written_files.push_back(".backup");
auto out = writeFileImpl(".backup");
std::unique_ptr<WriteBuffer> out;
if (use_archives)
out = getArchiveWriter("")->writeFile(".backup");
else
out = writer->writeFile(".backup");
out->write(str.data(), str.size());
}
void BackupImpl::readBackupMetadata()
{
auto in = readFileImpl(".backup");
std::unique_ptr<ReadBuffer> in;
if (use_archives)
in = getArchiveReader("")->readFile(".backup");
else
in = reader->readFile(".backup");
String str;
readStringUntilEOF(str, *in);
std::istringstream stream(str); // STYLE_CHECK_ALLOW_STD_STRING_STREAM
Poco::AutoPtr<Poco::Util::XMLConfiguration> config{new Poco::Util::XMLConfiguration()};
config->load(stream);
UInt64 version = config->getUInt("version");
if (version != BACKUP_VERSION)
throw Exception(ErrorCodes::BACKUP_VERSION_NOT_SUPPORTED, "Backup {}: Version {} is not supported", getName(), version);
version = config->getUInt("version");
if ((version < INITIAL_BACKUP_VERSION) || (version > CURRENT_BACKUP_VERSION))
throw Exception(ErrorCodes::BACKUP_VERSION_NOT_SUPPORTED, "Backup {}: Version {} is not supported", backup_name, version);
timestamp = parse<LocalDateTime>(config->getString("timestamp")).to_time_t();
uuid = parse<UUID>(config->getString("uuid"));
@ -244,7 +333,6 @@ void BackupImpl::readBackupMetadata()
if (config->has("base_backup_uuid"))
base_backup_uuid = parse<UUID>(config->getString("base_backup_uuid"));
file_infos.clear();
Poco::Util::AbstractConfiguration::Keys keys;
config->keys("contents", keys);
for (const auto & key : keys)
@ -252,99 +340,109 @@ void BackupImpl::readBackupMetadata()
if ((key == "file") || key.starts_with("file["))
{
String prefix = "contents." + key + ".";
String name = config->getString(prefix + "name");
FileInfo info;
info.file_name = config->getString(prefix + "name");
info.size = config->getUInt(prefix + "size");
if (info.size)
{
info.checksum = unhexChecksum(config->getString(prefix + "checksum"));
bool use_base = config->getBool(prefix + "use_base", false);
info.base_size = config->getUInt(prefix + "base_size", use_base ? info.size : 0);
if (info.base_size)
use_base = true;
if (info.base_size > info.size)
throw Exception(ErrorCodes::BACKUP_DAMAGED, "Backup {}: Base size must not be greater than the size of entry {}", backup_name, quoteString(info.file_name));
if (use_base)
{
if (info.base_size == info.size)
info.base_checksum = info.checksum;
else
info.base_checksum = unhexChecksum(config->getString(prefix + "base_checksum"));
}
if (info.size > info.base_size)
{
info.data_file_name = config->getString(prefix + "data_file", info.file_name);
info.archive_suffix = config->getString(prefix + "archive_suffix", "");
info.pos_in_archive = config->getUInt64(prefix + "pos_in_archive", static_cast<UInt64>(-1));
}
}
file_infos.emplace(name, info);
file_checksums.emplace(info.checksum, name);
coordination->addFileInfo(info);
}
}
}
Strings BackupImpl::listFiles(const String & prefix, const String & terminator) const
{
std::lock_guard lock{mutex};
if (!prefix.ends_with('/') && !prefix.empty())
throw Exception("prefix should end with '/'", ErrorCodes::BAD_ARGUMENTS);
std::lock_guard lock{mutex};
Strings elements;
for (auto it = file_infos.lower_bound(prefix); it != file_infos.end(); ++it)
{
const String & name = it->first;
if (!name.starts_with(prefix))
break;
size_t start_pos = prefix.length();
size_t end_pos = String::npos;
if (!terminator.empty())
end_pos = name.find(terminator, start_pos);
std::string_view new_element = std::string_view{name}.substr(start_pos, end_pos - start_pos);
if (!elements.empty() && (elements.back() == new_element))
continue;
elements.push_back(String{new_element});
}
return elements;
return coordination->listFiles(prefix, terminator);
}
bool BackupImpl::fileExists(const String & file_name) const
{
std::lock_guard lock{mutex};
return file_infos.contains(file_name);
return coordination->getFileInfo(file_name).has_value();
}
size_t BackupImpl::getFileSize(const String & file_name) const
bool BackupImpl::fileExists(const SizeAndChecksum & size_and_checksum) const
{
std::lock_guard lock{mutex};
auto it = file_infos.find(file_name);
if (it == file_infos.end())
return coordination->getFileInfo(size_and_checksum).has_value();
}
UInt64 BackupImpl::getFileSize(const String & file_name) const
{
std::lock_guard lock{mutex};
auto info = coordination->getFileInfo(file_name);
if (!info)
throw Exception(
ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Backup {}: Entry {} not found in the backup", getName(), quoteString(file_name));
return it->second.size;
ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Backup {}: Entry {} not found in the backup", backup_name, quoteString(file_name));
return info->size;
}
UInt128 BackupImpl::getFileChecksum(const String & file_name) const
{
std::lock_guard lock{mutex};
auto it = file_infos.find(file_name);
if (it == file_infos.end())
auto info = coordination->getFileInfo(file_name);
if (!info)
throw Exception(
ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Backup {}: Entry {} not found in the backup", getName(), quoteString(file_name));
return it->second.checksum;
ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Backup {}: Entry {} not found in the backup", backup_name, quoteString(file_name));
return info->checksum;
}
std::optional<String> BackupImpl::findFileByChecksum(const UInt128 & checksum) const
SizeAndChecksum BackupImpl::getFileSizeAndChecksum(const String & file_name) const
{
std::lock_guard lock{mutex};
auto it = file_checksums.find(checksum);
if (it == file_checksums.end())
return std::nullopt;
return it->second;
auto info = coordination->getFileInfo(file_name);
if (!info)
throw Exception(
ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Backup {}: Entry {} not found in the backup", backup_name, quoteString(file_name));
return std::pair(info->size, info->checksum);
}
BackupEntryPtr BackupImpl::readFile(const String & file_name) const
{
return readFile(getFileSizeAndChecksum(file_name));
}
BackupEntryPtr BackupImpl::readFile(const SizeAndChecksum & size_and_checksum) const
{
std::lock_guard lock{mutex};
if (open_mode != OpenMode::READ)
throw Exception("Backup is not opened for reading", ErrorCodes::LOGICAL_ERROR);
auto it = file_infos.find(file_name);
if (it == file_infos.end())
auto info_opt = coordination->getFileInfo(size_and_checksum);
if (!info_opt)
throw Exception(
ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Backup {}: Entry {} not found in the backup", getName(), quoteString(file_name));
ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Backup {}: Entry {} not found in the backup", backup_name, formatSizeAndChecksum(size_and_checksum));
const auto & info = it->second;
const auto & info = *info_opt;
if (!info.size)
{
/// Entry's data is empty.
@ -355,15 +453,7 @@ BackupEntryPtr BackupImpl::readFile(const String & file_name) const
{
/// Data goes completely from this backup, the base backup isn't used.
return std::make_unique<BackupEntryFromBackupImpl>(
std::static_pointer_cast<const BackupImpl>(shared_from_this()), file_name, info.size, info.checksum);
}
if (info.size < info.base_size)
{
throw Exception(
ErrorCodes::BACKUP_DAMAGED,
"Backup {}: Entry {} has its data size less than in the base backup {}: {} < {}",
getName(), quoteString(file_name), base_backup->getName(), info.size, info.base_size);
std::static_pointer_cast<const BackupImpl>(shared_from_this()), info.archive_suffix, info.data_file_name, info.size, info.checksum);
}
if (!base_backup)
@ -371,27 +461,18 @@ BackupEntryPtr BackupImpl::readFile(const String & file_name) const
throw Exception(
ErrorCodes::NO_BASE_BACKUP,
"Backup {}: Entry {} is marked to be read from a base backup, but there is no base backup specified",
getName(), quoteString(file_name));
backup_name, formatSizeAndChecksum(size_and_checksum));
}
auto base_file_name = base_backup->findFileByChecksum(info.base_checksum);
if (!base_file_name)
if (!base_backup->fileExists(std::pair(info.base_size, info.base_checksum)))
{
throw Exception(
ErrorCodes::WRONG_BASE_BACKUP,
"Backup {}: Entry {} is marked to be read from a base backup, but doesn't exist there",
getName(), quoteString(file_name));
backup_name, formatSizeAndChecksum(size_and_checksum));
}
auto base_entry = base_backup->readFile(*base_file_name);
auto base_size = base_entry->getSize();
if (base_size != info.base_size)
{
throw Exception(
ErrorCodes::WRONG_BASE_BACKUP,
"Backup {}: Entry {} has unexpected size in the base backup {}: {} (expected size: {})",
getName(), quoteString(file_name), base_backup->getName(), base_size, info.base_size);
}
auto base_entry = base_backup->readFile(std::pair{info.base_size, info.base_checksum});
if (info.size == info.base_size)
{
@ -399,10 +480,12 @@ BackupEntryPtr BackupImpl::readFile(const String & file_name) const
return base_entry;
}
/// The beginning of the data goes from the base backup,
/// and the ending goes from this backup.
return std::make_unique<BackupEntryFromBackupImpl>(
static_pointer_cast<const BackupImpl>(shared_from_this()), file_name, info.size, info.checksum, std::move(base_entry));
{
/// The beginning of the data goes from the base backup,
/// and the ending goes from this backup.
return std::make_unique<BackupEntryFromBackupImpl>(
static_pointer_cast<const BackupImpl>(shared_from_this()), info.archive_suffix, info.data_file_name, info.size, info.checksum, std::move(base_entry));
}
}
@ -412,17 +495,28 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry)
if (open_mode != OpenMode::WRITE)
throw Exception("Backup is not opened for writing", ErrorCodes::LOGICAL_ERROR);
if (file_infos.contains(file_name))
if (coordination->getFileInfo(file_name))
throw Exception(
ErrorCodes::BACKUP_ENTRY_ALREADY_EXISTS, "Backup {}: Entry {} already exists", getName(), quoteString(file_name));
ErrorCodes::BACKUP_ENTRY_ALREADY_EXISTS, "Backup {}: Entry {} already exists", backup_name, quoteString(file_name));
UInt64 size = entry->getSize();
std::optional<UInt128> checksum = entry->getChecksum();
FileInfo info;
info.file_name = file_name;
size_t size = entry->getSize();
info.size = size;
/// Check if the entry's data is empty.
if (!size)
if (!info.size)
{
file_infos.emplace(file_name, FileInfo{});
coordination->addFileInfo(info);
return;
}
/// Maybe we have a copy of this file in the backup already.
std::optional<UInt128> checksum = entry->getChecksum();
if (checksum && coordination->getFileInfo(std::pair{size, *checksum}))
{
info.checksum = *checksum;
coordination->addFileInfo(info);
return;
}
@ -436,7 +530,7 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry)
base_checksum = base_backup->getFileChecksum(file_name);
}
std::unique_ptr<ReadBuffer> read_buffer; /// We'll set that later.
std::unique_ptr<SeekableReadBuffer> read_buffer; /// We'll set that later.
std::optional<HashingReadBuffer> hashing_read_buffer;
UInt64 hashing_pos = 0; /// Current position in `hashing_read_buffer`.
@ -477,36 +571,45 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry)
checksum = hashing_read_buffer->getHash();
}
hashing_read_buffer.reset();
info.checksum = *checksum;
/// Maybe we have a copy of this file in the backup already.
if (coordination->getFileInfo(std::pair{size, *checksum}))
{
coordination->addFileInfo(info);
return;
}
/// Check if a entry with the same checksum exists in the base backup.
if (base_backup && !use_base)
if (base_backup && !use_base && base_backup->fileExists(std::pair{size, *checksum}))
{
if (auto base_file_name = base_backup->findFileByChecksum(*checksum))
{
if (size == base_backup->getFileSize(*base_file_name))
{
/// The entry's data has not changed since the base backup,
/// but the entry itself has been moved or renamed.
base_size = size;
base_checksum = *checksum;
use_base = true;
}
}
/// The entry's data has not changed since the base backup,
/// but the entry itself has been moved or renamed.
base_size = size;
base_checksum = *checksum;
use_base = true;
}
if (use_base)
{
info.base_size = base_size;
info.base_checksum = base_checksum;
}
if (use_base && (size == base_size))
{
/// The entry's data has not been changed since the base backup.
FileInfo info;
info.size = size;
info.checksum = *checksum;
info.base_size = base_size;
info.base_checksum = base_checksum;
file_infos.emplace(file_name, info);
file_checksums.emplace(*checksum, file_name);
coordination->addFileInfo(info);
return;
}
bool is_data_file_required;
info.data_file_name = info.file_name;
info.archive_suffix = current_archive_suffix;
coordination->addFileInfo(info, is_data_file_required);
if (!is_data_file_required)
return; /// We copy data only if it's a new combination of size & checksum.
/// Either the entry wasn't exist in the base backup
/// or the entry has data appended to the end of the data from the base backup.
/// In both those cases we have to copy data to this backup.
@ -515,47 +618,107 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry)
auto copy_pos = use_base ? base_size : 0;
/// Move the current read position to the start position to copy data.
/// If `read_buffer` is seekable it's easier, otherwise we can use ignore().
if (auto * seekable_buffer = dynamic_cast<SeekableReadBuffer *>(read_buffer.get()))
if (!read_buffer)
read_buffer = entry->getReadBuffer();
read_buffer->seek(copy_pos, SEEK_SET);
/// Copy the entry's data after `copy_pos`.
std::unique_ptr<WriteBuffer> out;
if (use_archives)
{
seekable_buffer->seek(copy_pos, SEEK_SET);
String archive_suffix = current_archive_suffix;
bool next_suffix = false;
if (current_archive_suffix.empty() && is_internal_backup)
next_suffix = true;
/*if (archive_params.max_volume_size && current_archive_writer
&& (current_archive_writer->getTotalSize() + size - base_size > archive_params.max_volume_size))
next_suffix = true;*/
if (next_suffix)
current_archive_suffix = coordination->getNextArchiveSuffix();
if (info.archive_suffix != current_archive_suffix)
{
info.archive_suffix = current_archive_suffix;
coordination->updateFileInfo(info);
}
out = getArchiveWriter(current_archive_suffix)->writeFile(info.data_file_name);
}
else
{
read_buffer = entry->getReadBuffer();
read_buffer->ignore(copy_pos);
out = writer->writeFile(info.data_file_name);
}
/// Copy the entry's data after `copy_pos`.
written_files.push_back(file_name);
auto out = writeFileImpl(file_name);
copyData(*read_buffer, *out);
/// Done!
FileInfo info;
info.size = size;
info.checksum = *checksum;
if (use_base)
{
info.base_size = base_size;
info.base_checksum = base_checksum;
}
file_infos.emplace(file_name, info);
file_checksums.emplace(*checksum, file_name);
}
void BackupImpl::finalizeWriting()
{
std::lock_guard lock{mutex};
if (writing_finalized)
return;
if (open_mode != OpenMode::WRITE)
throw Exception("Backup is not opened for writing", ErrorCodes::LOGICAL_ERROR);
writeBackupMetadata();
writing_finalized = true;
}
String BackupImpl::getArchiveNameWithSuffix(const String & suffix) const
{
return archive_params.archive_name + (suffix.empty() ? "" : ".") + suffix;
}
std::shared_ptr<IArchiveReader> BackupImpl::getArchiveReader(const String & suffix) const
{
auto it = archive_readers.find(suffix);
if (it != archive_readers.end())
return it->second;
String archive_name_with_suffix = getArchiveNameWithSuffix(suffix);
size_t archive_size = reader->getFileSize(archive_name_with_suffix);
auto new_archive_reader = createArchiveReader(archive_params.archive_name, [reader=reader, archive_name_with_suffix]{ return reader->readFile(archive_name_with_suffix); },
archive_size);
new_archive_reader->setPassword(archive_params.password);
archive_readers.emplace(suffix, new_archive_reader);
return new_archive_reader;
}
std::shared_ptr<IArchiveWriter> BackupImpl::getArchiveWriter(const String & suffix)
{
for (const auto & archive_writer : archive_writers)
{
if ((suffix == archive_writer.first) && archive_writer.second)
return archive_writer.second;
}
String archive_name_with_suffix = getArchiveNameWithSuffix(suffix);
auto new_archive_writer = createArchiveWriter(archive_params.archive_name, writer->writeFile(archive_name_with_suffix));
new_archive_writer->setPassword(archive_params.password);
size_t pos = suffix.empty() ? 0 : 1;
archive_writers[pos] = {suffix, new_archive_writer};
return new_archive_writer;
}
void BackupImpl::removeAllFilesAfterFailure()
{
Strings files_to_remove;
if (use_archives)
{
files_to_remove.push_back(archive_params.archive_name);
for (const auto & suffix : coordination->getAllArchiveSuffixes())
{
String archive_name_with_suffix = getArchiveNameWithSuffix(suffix);
files_to_remove.push_back(std::move(archive_name_with_suffix));
}
}
else
{
files_to_remove.push_back(".backup");
for (const auto & file_info : coordination->getAllFileInfos())
files_to_remove.push_back(file_info.data_file_name);
}
writer->removeFilesAfterFailure(files_to_remove);
}
}

View File

@ -1,92 +1,107 @@
#pragma once
#include <Backups/IBackup.h>
#include <Backups/IBackupCoordination.h>
#include <Backups/BackupInfo.h>
#include <map>
#include <mutex>
#include <unordered_map>
namespace DB
{
class IBackupCoordination;
class IBackupReader;
class IBackupWriter;
class SeekableReadBuffer;
class IArchiveReader;
class IArchiveWriter;
class Context;
using ContextPtr = std::shared_ptr<const Context>;
/// Base implementation of IBackup.
/// Implementation of IBackup.
/// Along with passed files it also stores backup metadata - a single file named ".backup" in XML format
/// which contains a list of all files in the backup with their sizes and checksums and information
/// whether the base backup should be used for each entry.
class BackupImpl : public IBackup
{
public:
struct ArchiveParams
{
String archive_name;
String password;
String compression_method;
int compression_level = 0;
size_t max_volume_size = 0;
};
BackupImpl(
const String & backup_name_,
const ArchiveParams & archive_params_,
const std::optional<BackupInfo> & base_backup_info_,
std::shared_ptr<IBackupReader> reader_,
const ContextPtr & context_);
BackupImpl(
const String & backup_name_,
const ArchiveParams & archive_params_,
const std::optional<BackupInfo> & base_backup_info_,
std::shared_ptr<IBackupWriter> writer_,
const ContextPtr & context_,
const std::optional<BackupInfo> & base_backup_info_ = {});
const std::optional<UUID> & backup_uuid_ = {},
bool is_internal_backup_ = false,
const String & coordination_zk_path_ = {});
~BackupImpl() override;
const String & getName() const override { return backup_name; }
void open(OpenMode open_mode_) override;
OpenMode getOpenMode() const override;
void close() override;
OpenMode getOpenMode() const override { return open_mode; }
time_t getTimestamp() const override;
UUID getUUID() const override { return uuid; }
UUID getUUID() const override { return *uuid; }
Strings listFiles(const String & prefix, const String & terminator) const override;
bool fileExists(const String & file_name) const override;
size_t getFileSize(const String & file_name) const override;
bool fileExists(const SizeAndChecksum & size_and_checksum) const override;
UInt64 getFileSize(const String & file_name) const override;
UInt128 getFileChecksum(const String & file_name) const override;
std::optional<String> findFileByChecksum(const UInt128 & checksum) const override;
SizeAndChecksum getFileSizeAndChecksum(const String & file_name) const override;
BackupEntryPtr readFile(const String & file_name) const override;
BackupEntryPtr readFile(const SizeAndChecksum & size_and_checksum) const override;
void writeFile(const String & file_name, BackupEntryPtr entry) override;
void finalizeWriting() override;
protected:
/// Checks if this backup exists.
virtual bool backupExists() const = 0;
virtual void openImpl(OpenMode open_mode_) = 0;
OpenMode getOpenModeNoLock() const { return open_mode; }
virtual void closeImpl(const Strings & written_files_, bool writing_finalized_) = 0;
/// Read a file from the backup.
/// Low level: the function doesn't check base backup or checksums.
virtual std::unique_ptr<ReadBuffer> readFileImpl(const String & file_name) const = 0;
/// Add a file to the backup.
/// Low level: the function doesn't check base backup or checksums.
virtual std::unique_ptr<WriteBuffer> writeFileImpl(const String & file_name) = 0;
mutable std::mutex mutex;
bool supportsWritingInMultipleThreads() const override { return !use_archives; }
private:
void writeBackupMetadata();
void readBackupMetadata();
struct FileInfo
{
UInt64 size = 0;
UInt128 checksum{0, 0};
/// for incremental backups
UInt64 base_size = 0;
UInt128 base_checksum{0, 0};
};
using FileInfo = IBackupCoordination::FileInfo;
class BackupEntryFromBackupImpl;
void open();
void close();
void writeBackupMetadata();
void readBackupMetadata();
String getArchiveNameWithSuffix(const String & suffix) const;
std::shared_ptr<IArchiveReader> getArchiveReader(const String & suffix) const;
std::shared_ptr<IArchiveWriter> getArchiveWriter(const String & suffix);
void removeAllFilesAfterFailure();
const String backup_name;
const ArchiveParams archive_params;
const bool use_archives;
const OpenMode open_mode;
std::shared_ptr<IBackupWriter> writer;
std::shared_ptr<IBackupReader> reader;
const bool is_internal_backup;
std::shared_ptr<IBackupCoordination> coordination;
ContextPtr context;
const std::optional<BackupInfo> base_backup_info_param;
OpenMode open_mode = OpenMode::NONE;
UUID uuid = {};
mutable std::mutex mutex;
std::optional<UUID> uuid;
time_t timestamp = 0;
UInt64 version;
std::optional<BackupInfo> base_backup_info;
std::shared_ptr<const IBackup> base_backup;
std::optional<UUID> base_backup_uuid;
std::map<String, FileInfo> file_infos; /// Should be ordered alphabetically, see listFiles().
std::unordered_map<UInt128, String> file_checksums;
Strings written_files;
mutable std::unordered_map<String /* archive_suffix */, std::shared_ptr<IArchiveReader>> archive_readers;
std::pair<String, std::shared_ptr<IArchiveWriter>> archive_writers[2];
String current_archive_suffix;
bool writing_finalized = false;
};

View File

@ -16,6 +16,21 @@ namespace ErrorCodes
}
String BackupInfo::toString() const
{
ASTPtr ast = toAST();
return serializeAST(*ast);
}
BackupInfo BackupInfo::fromString(const String & str)
{
ParserIdentifierWithOptionalParameters parser;
ASTPtr ast = parseQuery(parser, str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
return fromAST(*ast);
}
ASTPtr BackupInfo::toAST() const
{
auto func = std::make_shared<ASTFunction>();
func->name = backup_engine_name;
@ -32,15 +47,7 @@ String BackupInfo::toString() const
for (const auto & arg : args)
list->children.push_back(std::make_shared<ASTLiteral>(arg));
return serializeAST(*func);
}
BackupInfo BackupInfo::fromString(const String & str)
{
ParserIdentifierWithOptionalParameters parser;
ASTPtr ast = parseQuery(parser, str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
return fromAST(*ast);
return func;
}

View File

@ -6,6 +6,7 @@
namespace DB
{
class IAST;
using ASTPtr = std::shared_ptr<IAST>;
/// Information about a backup.
struct BackupInfo
@ -16,6 +17,8 @@ struct BackupInfo
String toString() const;
static BackupInfo fromString(const String & str);
ASTPtr toAST() const;
static BackupInfo fromAST(const IAST & ast);
};

View File

@ -1,5 +1,5 @@
#include <Backups/BackupSettings.h>
#include <Backups/BackupInfo.h>
#include <Backups/BackupSettings.h>
#include <Core/SettingsFields.h>
#include <Parsers/ASTBackupQuery.h>
#include <Parsers/ASTSetQuery.h>
@ -9,9 +9,23 @@ namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_SETTING;
extern const int CANNOT_PARSE_BACKUP_SETTINGS;
}
/// List of backup settings except base_backup_name.
#define LIST_OF_BACKUP_SETTINGS(M) \
M(String, compression_method) \
M(Int64, compression_level) \
M(String, password) \
M(Bool, structure_only) \
M(Bool, async) \
M(UInt64, shard_num) \
M(UInt64, replica_num) \
M(Bool, allow_storing_multiple_replicas) \
M(Bool, internal) \
M(String, coordination_zk_path)
BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query)
{
BackupSettings res;
@ -24,20 +38,35 @@ BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query)
const auto & settings = query.settings->as<const ASTSetQuery &>().changes;
for (const auto & setting : settings)
{
if (setting.name == "compression_method")
res.compression_method = SettingFieldString{setting.value};
else if (setting.name == "compression_level")
res.compression_level = SettingFieldInt64{setting.value};
else if (setting.name == "password")
res.password = SettingFieldString{setting.value};
else if (setting.name == "structure_only")
res.structure_only = SettingFieldBool{setting.value};
#define GET_SETTINGS_FROM_BACKUP_QUERY_HELPER(TYPE, NAME) \
if (setting.name == #NAME) \
res.NAME = SettingField##TYPE{setting.value}.value; \
else
throw Exception(ErrorCodes::UNKNOWN_SETTING, "Unknown setting {}", setting.name);
LIST_OF_BACKUP_SETTINGS(GET_SETTINGS_FROM_BACKUP_QUERY_HELPER)
throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.name);
}
}
return res;
}
void BackupSettings::copySettingsToBackupQuery(ASTBackupQuery & query) const
{
query.base_backup_name = base_backup_info ? base_backup_info->toAST() : nullptr;
auto query_settings = std::make_shared<ASTSetQuery>();
query_settings->is_standalone = false;
static const BackupSettings default_settings;
#define SET_SETTINGS_IN_BACKUP_QUERY_HELPER(TYPE, NAME) \
if ((NAME) != default_settings.NAME) \
query_settings->changes.emplace_back(#NAME, static_cast<Field>(SettingField##TYPE{NAME}));
LIST_OF_BACKUP_SETTINGS(SET_SETTINGS_IN_BACKUP_QUERY_HELPER)
query.settings = query_settings;
}
}

View File

@ -25,7 +25,30 @@ struct BackupSettings
/// without the data of tables.
bool structure_only = false;
/// Whether the BACKUP command must return immediately without waiting until the backup has completed.
bool async = false;
/// 1-based shard index to store in the backup. 0 means all shards.
/// Can only be used with BACKUP ON CLUSTER.
size_t shard_num = 0;
/// 1-based replica index to store in the backup. 0 means all replicas (see also allow_storing_multiple_replicas).
/// Can only be used with BACKUP ON CLUSTER.
size_t replica_num = 0;
/// Allows storing in the backup of multiple replicas.
bool allow_storing_multiple_replicas = false;
/// Internal, should not be specified by user.
/// Whether this backup is a part of a distributed backup created by BACKUP ON CLUSTER.
bool internal = false;
/// Internal, should not be specified by user.
/// Path in Zookeeper used to coordinate a distributed backup created by BACKUP ON CLUSTER.
String coordination_zk_path;
static BackupSettings fromBackupQuery(const ASTBackupQuery & query);
void copySettingsToBackupQuery(ASTBackupQuery & query) const;
};
}

View File

@ -0,0 +1,50 @@
#include <Backups/BackupStatus.h>
#include <Common/Exception.h>
#include <base/range.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
std::string_view toString(BackupStatus backup_status)
{
switch (backup_status)
{
case BackupStatus::PREPARING:
return "PREPARING";
case BackupStatus::MAKING_BACKUP:
return "MAKING_BACKUP";
case BackupStatus::BACKUP_COMPLETE:
return "BACKUP_COMPLETE";
case BackupStatus::FAILED_TO_BACKUP:
return "FAILED_TO_BACKUP";
case BackupStatus::RESTORING:
return "RESTORING";
case BackupStatus::RESTORED:
return "RESTORED";
case BackupStatus::FAILED_TO_RESTORE:
return "FAILED_TO_RESTORE";
default:
break;
}
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected backup status: {}", static_cast<int>(backup_status));
}
const std::vector<std::pair<String, Int8>> & getBackupStatusEnumValues()
{
static const std::vector<std::pair<String, Int8>> values = []
{
std::vector<std::pair<String, Int8>> res;
for (auto status : collections::range(BackupStatus::MAX))
res.emplace_back(toString(status), static_cast<Int8>(status));
return res;
}();
return values;
}
}

View File

@ -0,0 +1,31 @@
#pragma once
#include <base/types.h>
namespace DB
{
enum class BackupStatus
{
/// Statuses of making backups
PREPARING,
MAKING_BACKUP,
BACKUP_COMPLETE,
FAILED_TO_BACKUP,
/// Status of restoring
RESTORING,
RESTORED,
FAILED_TO_RESTORE,
MAX,
};
std::string_view toString(BackupStatus backup_status);
/// Returns vector containing all values of BackupStatus and their string representation,
/// which is used to create DataTypeEnum8 to store those statuses.
const std::vector<std::pair<String, Int8>> & getBackupStatusEnumValues();
}

View File

@ -5,11 +5,13 @@
#include <Backups/DDLRenamingVisitor.h>
#include <Backups/IBackup.h>
#include <Backups/formatTableNameOrTemporaryTableName.h>
#include <Backups/replaceTableUUIDWithMacroInReplicatedTableDef.h>
#include <Common/escapeForFileName.h>
#include <Access/Common/AccessFlags.h>
#include <Databases/IDatabase.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/formatAST.h>
#include <Storages/IStorage.h>
@ -26,6 +28,67 @@ namespace ErrorCodes
namespace
{
/// Helper to calculate paths inside a backup.
class PathsInBackup
{
public:
/// Returns the path to metadata in backup.
static String getMetadataPath(const DatabaseAndTableName & table_name, size_t shard_index, size_t replica_index)
{
if (table_name.first.empty() || table_name.second.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name and table name must not be empty");
return getPathForShardAndReplica(shard_index, replica_index) + String{"metadata/"} + escapeForFileName(table_name.first) + "/"
+ escapeForFileName(table_name.second) + ".sql";
}
static String getMetadataPath(const String & database_name, size_t shard_index, size_t replica_index)
{
if (database_name.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name must not be empty");
return getPathForShardAndReplica(shard_index, replica_index) + String{"metadata/"} + escapeForFileName(database_name) + ".sql";
}
static String getMetadataPath(const IAST & create_query, size_t shard_index, size_t replica_index)
{
const auto & create = create_query.as<const ASTCreateQuery &>();
if (!create.table)
return getMetadataPath(create.getDatabase(), shard_index, replica_index);
if (create.temporary)
return getMetadataPath({DatabaseCatalog::TEMPORARY_DATABASE, create.getTable()}, shard_index, replica_index);
return getMetadataPath({create.getDatabase(), create.getTable()}, shard_index, replica_index);
}
/// Returns the path to table's data in backup.
static String getDataPath(const DatabaseAndTableName & table_name, size_t shard_index, size_t replica_index)
{
if (table_name.first.empty() || table_name.second.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name and table name must not be empty");
assert(!table_name.first.empty() && !table_name.second.empty());
return getPathForShardAndReplica(shard_index, replica_index) + String{"data/"} + escapeForFileName(table_name.first) + "/"
+ escapeForFileName(table_name.second) + "/";
}
static String getDataPath(const IAST & create_query, size_t shard_index, size_t replica_index)
{
const auto & create = create_query.as<const ASTCreateQuery &>();
if (!create.table)
return {};
if (create.temporary)
return getDataPath({DatabaseCatalog::TEMPORARY_DATABASE, create.getTable()}, shard_index, replica_index);
return getDataPath({create.getDatabase(), create.getTable()}, shard_index, replica_index);
}
private:
static String getPathForShardAndReplica(size_t shard_index, size_t replica_index)
{
if (shard_index || replica_index)
return fmt::format("shards/{}/replicas/{}/", shard_index, replica_index);
else
return "";
}
};
using Kind = ASTBackupQuery::Kind;
using Element = ASTBackupQuery::Element;
using Elements = ASTBackupQuery::Elements;
@ -80,15 +143,6 @@ namespace
/// Makes backup entries, should be called after prepare().
BackupEntries makeBackupEntries() const
{
/// Check that there are not `different_create_query`. (If it's set it means error.)
for (const auto & info : databases | boost::adaptors::map_values)
{
if (info.different_create_query)
throw Exception(ErrorCodes::CANNOT_BACKUP_DATABASE,
"Cannot backup a database because two different create queries were generated for it: {} and {}",
serializeAST(*info.create_query), serializeAST(*info.different_create_query));
}
BackupEntries res;
for (const auto & info : databases | boost::adaptors::map_values)
res.push_back(makeBackupEntryForMetadata(*info.create_query));
@ -101,7 +155,7 @@ namespace
auto data_backup = info.storage->backupData(context, info.partitions);
if (!data_backup.empty())
{
String data_path = getDataPathInBackup(*info.create_query);
String data_path = PathsInBackup::getDataPath(*info.create_query, backup_settings.shard_num, backup_settings.replica_num);
for (auto & [path_in_backup, backup_entry] : data_backup)
res.emplace_back(data_path + path_in_backup, std::move(backup_entry));
}
@ -138,9 +192,9 @@ namespace
database->getEngineName());
/// Check that we are not trying to backup the same table again.
DatabaseAndTableName new_table_name = renaming_settings.getNewTableName(table_name_);
if (tables.contains(new_table_name))
throw Exception(ErrorCodes::CANNOT_BACKUP_TABLE, "Cannot backup the {} twice", formatTableNameOrTemporaryTableName(new_table_name));
DatabaseAndTableName name_in_backup = renaming_settings.getNewTableName(table_name_);
if (tables.contains(name_in_backup))
throw Exception(ErrorCodes::CANNOT_BACKUP_TABLE, "Cannot backup the {} twice", formatTableNameOrTemporaryTableName(name_in_backup));
/// Make a create query for this table.
auto create_query = prepareCreateQueryForBackup(database->getCreateTableQuery(table_name_.second, context));
@ -155,40 +209,9 @@ namespace
CreateTableInfo info;
info.create_query = create_query;
info.storage = storage;
info.name_in_backup = new_table_name;
info.partitions = partitions_;
info.has_data = has_data;
tables[new_table_name] = std::move(info);
/// If it's not system or temporary database then probably we need to backup the database's definition too.
if (!isSystemOrTemporaryDatabase(table_name_.first))
{
if (!databases.contains(new_table_name.first))
{
/// Add a create query to backup the database if we haven't done it yet.
auto create_db_query = prepareCreateQueryForBackup(database->getCreateDatabaseQuery());
create_db_query->setDatabase(new_table_name.first);
CreateDatabaseInfo info_db;
info_db.create_query = create_db_query;
info_db.original_name = table_name_.first;
info_db.is_explicit = false;
databases[new_table_name.first] = std::move(info_db);
}
else
{
/// We already have added a create query to backup the database,
/// set `different_create_query` if it's not the same.
auto & info_db = databases[new_table_name.first];
if (!info_db.is_explicit && (info_db.original_name != table_name_.first) && !info_db.different_create_query)
{
auto create_db_query = prepareCreateQueryForBackup(table_.first->getCreateDatabaseQuery());
create_db_query->setDatabase(new_table_name.first);
if (!areDatabaseDefinitionsSame(*info_db.create_query, *create_db_query))
info_db.different_create_query = create_db_query;
}
}
}
tables[name_in_backup] = std::move(info);
}
/// Prepares to restore a database and all tables in it.
@ -203,21 +226,19 @@ namespace
context->checkAccess(AccessType::SHOW_DATABASES, database_name_);
/// Check that we are not trying to restore the same database again.
String new_database_name = renaming_settings.getNewDatabaseName(database_name_);
if (databases.contains(new_database_name) && databases[new_database_name].is_explicit)
throw Exception(ErrorCodes::CANNOT_BACKUP_DATABASE, "Cannot backup the database {} twice", backQuoteIfNeed(new_database_name));
String name_in_backup = renaming_settings.getNewDatabaseName(database_name_);
if (databases.contains(name_in_backup))
throw Exception(ErrorCodes::CANNOT_BACKUP_DATABASE, "Cannot backup the database {} twice", backQuoteIfNeed(name_in_backup));
/// Of course we're not going to backup the definition of the system or the temporary database.
if (!isSystemOrTemporaryDatabase(database_name_))
{
/// Make a create query for this database.
auto create_db_query = prepareCreateQueryForBackup(database_->getCreateDatabaseQuery());
auto create_query = prepareCreateQueryForBackup(database_->getCreateDatabaseQuery());
CreateDatabaseInfo info_db;
info_db.create_query = create_db_query;
info_db.original_name = database_name_;
info_db.is_explicit = true;
databases[new_database_name] = std::move(info_db);
CreateDatabaseInfo info;
info.create_query = create_query;
databases[name_in_backup] = std::move(info);
}
/// Backup tables in this database.
@ -251,6 +272,7 @@ namespace
ASTPtr query = ast;
::DB::renameInCreateQuery(query, context, renaming_settings);
auto create_query = typeid_cast<std::shared_ptr<ASTCreateQuery>>(query);
replaceTableUUIDWithMacroInReplicatedTableDef(*create_query, create_query->uuid);
create_query->uuid = UUIDHelpers::Nil;
create_query->to_inner_uuid = UUIDHelpers::Nil;
return create_query;
@ -261,10 +283,10 @@ namespace
return (database_name == DatabaseCatalog::SYSTEM_DATABASE) || (database_name == DatabaseCatalog::TEMPORARY_DATABASE);
}
static std::pair<String, BackupEntryPtr> makeBackupEntryForMetadata(const IAST & create_query)
std::pair<String, BackupEntryPtr> makeBackupEntryForMetadata(const IAST & create_query) const
{
auto metadata_entry = std::make_unique<BackupEntryFromMemory>(serializeAST(create_query));
String metadata_path = getMetadataPathInBackup(create_query);
String metadata_path = PathsInBackup::getMetadataPath(create_query, backup_settings.shard_num, backup_settings.replica_num);
return {metadata_path, std::move(metadata_entry)};
}
@ -273,7 +295,6 @@ namespace
{
ASTPtr create_query;
StoragePtr storage;
DatabaseAndTableName name_in_backup;
ASTs partitions;
bool has_data = false;
};
@ -282,24 +303,13 @@ namespace
struct CreateDatabaseInfo
{
ASTPtr create_query;
String original_name;
/// Whether the creation of this database is specified explicitly, via RESTORE DATABASE or
/// RESTORE ALL DATABASES.
/// It's false if the creation of this database is caused by creating a table contained in it.
bool is_explicit = false;
/// If this is set it means the following error:
/// it means that for implicitly created database there were two different create query
/// generated so we cannot restore the database.
ASTPtr different_create_query;
};
ContextPtr context;
BackupSettings backup_settings;
DDLRenamingSettings renaming_settings;
std::map<String, CreateDatabaseInfo> databases;
std::map<DatabaseAndTableName, CreateTableInfo> tables;
std::unordered_map<String /* db_name_in_backup */, CreateDatabaseInfo> databases;
std::map<DatabaseAndTableName /* table_name_in_backup */, CreateTableInfo> tables;
};
}
@ -373,47 +383,4 @@ void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries
backup->finalizeWriting();
}
String getDataPathInBackup(const DatabaseAndTableName & table_name)
{
if (table_name.first.empty() || table_name.second.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name and table name must not be empty");
assert(!table_name.first.empty() && !table_name.second.empty());
return String{"data/"} + escapeForFileName(table_name.first) + "/" + escapeForFileName(table_name.second) + "/";
}
String getDataPathInBackup(const IAST & create_query)
{
const auto & create = create_query.as<const ASTCreateQuery &>();
if (!create.table)
return {};
if (create.temporary)
return getDataPathInBackup({DatabaseCatalog::TEMPORARY_DATABASE, create.getTable()});
return getDataPathInBackup({create.getDatabase(), create.getTable()});
}
String getMetadataPathInBackup(const DatabaseAndTableName & table_name)
{
if (table_name.first.empty() || table_name.second.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name and table name must not be empty");
return String{"metadata/"} + escapeForFileName(table_name.first) + "/" + escapeForFileName(table_name.second) + ".sql";
}
String getMetadataPathInBackup(const String & database_name)
{
if (database_name.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name must not be empty");
return String{"metadata/"} + escapeForFileName(database_name) + ".sql";
}
String getMetadataPathInBackup(const IAST & create_query)
{
const auto & create = create_query.as<const ASTCreateQuery &>();
if (!create.table)
return getMetadataPathInBackup(create.getDatabase());
if (create.temporary)
return getMetadataPathInBackup({DatabaseCatalog::TEMPORARY_DATABASE, create.getTable()});
return getMetadataPathInBackup({create.getDatabase(), create.getTable()});
}
}

View File

@ -6,6 +6,7 @@
namespace DB
{
class IBackup;
using BackupPtr = std::shared_ptr<const IBackup>;
using BackupMutablePtr = std::shared_ptr<IBackup>;
class IBackupEntry;
using BackupEntryPtr = std::unique_ptr<IBackupEntry>;
@ -20,13 +21,4 @@ BackupEntries makeBackupEntries(const ContextPtr & context, const ASTBackupQuery
/// Write backup entries to an opened backup.
void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries, size_t num_threads);
/// Returns the path to metadata in backup.
String getMetadataPathInBackup(const DatabaseAndTableName & table_name);
String getMetadataPathInBackup(const String & database_name);
String getMetadataPathInBackup(const IAST & create_query);
/// Returns the path to table's data in backup.
String getDataPathInBackup(const DatabaseAndTableName & table_name);
String getDataPathInBackup(const IAST & create_query);
}

View File

@ -0,0 +1,93 @@
#include <Backups/BackupsWorker.h>
#include <Common/Exception.h>
#include <base/logger_useful.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
BackupsWorker & BackupsWorker::instance()
{
static BackupsWorker the_instance;
return the_instance;
}
BackupsWorker::BackupsWorker() = default;
size_t BackupsWorker::add(const String & backup_name, BackupStatus status, const String & error)
{
std::lock_guard lock{mutex};
size_t task_id = ++current_task_id;
size_t pos;
auto it = entries_by_name.find(backup_name);
if (it != entries_by_name.end())
{
pos = it->second;
entries_by_task_id.erase(entries[pos].task_id);
}
else
{
pos = entries.size();
entries.emplace_back().backup_name = backup_name;
entries_by_name.emplace(backup_name, pos);
}
entries_by_task_id.emplace(task_id, pos);
Entry & entry = entries[pos];
entry.task_id = task_id;
entry.status = status;
entry.error = error;
entry.timestamp = std::time(nullptr);
return task_id;
}
void BackupsWorker::update(size_t task_id, BackupStatus status, const String & error)
{
std::lock_guard lock{mutex};
auto it = entries_by_task_id.find(task_id);
if ((it == entries_by_task_id.end()) || (it->second >= entries.size()))
throw Exception(ErrorCodes::LOGICAL_ERROR, "BackupsWorker: entry_id is out of range");
Entry & entry = entries[it->second];
entry.status = status;
entry.error = error;
entry.timestamp = std::time(nullptr);
}
BackupsWorker::Entry BackupsWorker::getEntry(size_t task_id) const
{
std::lock_guard lock{mutex};
auto it = entries_by_task_id.find(task_id);
if ((it == entries_by_task_id.end()) || (it->second >= entries.size()))
throw Exception(ErrorCodes::LOGICAL_ERROR, "BackupsWorker: entry_id is out of range");
return entries[it->second];
}
std::vector<BackupsWorker::Entry> BackupsWorker::getEntries() const
{
std::lock_guard lock{mutex};
return entries;
}
void BackupsWorker::run(std::function<void()> && task)
{
thread_pool.scheduleOrThrowOnError(std::move(task));
}
void BackupsWorker::shutdown()
{
size_t num_active_tasks = thread_pool.active();
if (!num_active_tasks)
return;
LOG_INFO(&Poco::Logger::get("BackupsWorker"), "Waiting for {} backup or restore tasks to be finished", num_active_tasks);
thread_pool.wait();
LOG_INFO(&Poco::Logger::get("BackupsWorker"), "All backup and restore tasks have finished");
}
}

View File

@ -0,0 +1,51 @@
#pragma once
#include <Backups/BackupStatus.h>
#include <Common/ThreadPool.h>
#include <Core/UUID.h>
#include <unordered_map>
namespace DB
{
/// Manager of backups and restores: executes backups and restores' threads in the background.
/// Keeps information about backups and restores started in this session.
class BackupsWorker
{
public:
static BackupsWorker & instance();
size_t add(const String & backup_name, BackupStatus status, const String & error = {});
void update(size_t task_id, BackupStatus status, const String & error = {});
struct Entry
{
String backup_name;
size_t task_id;
BackupStatus status;
String error;
time_t timestamp;
};
Entry getEntry(size_t task_id) const;
std::vector<Entry> getEntries() const;
/// Schedules a new task and performs it in the background thread.
void run(std::function<void()> && task);
/// Waits until all tasks have been completed.
void shutdown();
private:
BackupsWorker();
mutable std::mutex mutex;
std::vector<Entry> entries;
std::unordered_map<String, size_t /* position in entries */> entries_by_name;
std::unordered_map<size_t /* task_id */, size_t /* position in entries */ > entries_by_task_id;
size_t current_task_id = 0;
ThreadPool thread_pool;
};
}

View File

@ -1,72 +0,0 @@
#include <Backups/DirectoryBackup.h>
#include <Disks/DiskLocal.h>
namespace DB
{
DirectoryBackup::DirectoryBackup(
const String & backup_name_,
const DiskPtr & disk_,
const String & path_,
const ContextPtr & context_,
const std::optional<BackupInfo> & base_backup_info_)
: BackupImpl(backup_name_, context_, base_backup_info_)
, disk(disk_)
{
/// Remove terminating slash.
path = (std::filesystem::path(path_) / "").parent_path();
/// If `disk` is not specified, we create an internal instance of `DiskLocal` here.
if (!disk)
{
disk = std::make_shared<DiskLocal>(path, path, 0);
path = ".";
}
}
DirectoryBackup::~DirectoryBackup()
{
close();
}
bool DirectoryBackup::backupExists() const
{
return disk->isDirectory(path);
}
void DirectoryBackup::openImpl(OpenMode open_mode_)
{
if (open_mode_ == OpenMode::WRITE)
disk->createDirectories(path);
}
void DirectoryBackup::closeImpl(const Strings & written_files_, bool writing_finalized_)
{
if ((getOpenModeNoLock() == OpenMode::WRITE) && !writing_finalized_ && !written_files_.empty())
{
/// Creating of the backup wasn't finished correctly,
/// so the backup cannot be used and it's better to remove its files.
const auto & files_to_delete = written_files_;
for (const String & file_name : files_to_delete)
disk->removeFileIfExists(path / file_name);
if (disk->isDirectory(path) && disk->isDirectoryEmpty(path))
disk->removeDirectory(path);
}
}
std::unique_ptr<ReadBuffer> DirectoryBackup::readFileImpl(const String & file_name) const
{
auto file_path = path / file_name;
return disk->readFile(file_path);
}
std::unique_ptr<WriteBuffer> DirectoryBackup::writeFileImpl(const String & file_name)
{
auto file_path = path / file_name;
disk->createDirectories(fs::path(file_path).parent_path());
return disk->writeFile(file_path);
}
}

View File

@ -1,37 +0,0 @@
#pragma once
#include <Backups/BackupImpl.h>
#include <filesystem>
namespace DB
{
class IDisk;
using DiskPtr = std::shared_ptr<IDisk>;
/// Represents a backup stored on a disk.
/// A backup is stored as a directory, each entry is stored as a file in that directory.
class DirectoryBackup : public BackupImpl
{
public:
/// `disk`_ is allowed to be nullptr and that means the `path_` is a path in the local filesystem.
DirectoryBackup(
const String & backup_name_,
const DiskPtr & disk_,
const String & path_,
const ContextPtr & context_,
const std::optional<BackupInfo> & base_backup_info_ = {});
~DirectoryBackup() override;
private:
bool backupExists() const override;
void openImpl(OpenMode open_mode_) override;
void closeImpl(const Strings & written_files_, bool writing_finalized_) override;
std::unique_ptr<ReadBuffer> readFileImpl(const String & file_name) const override;
std::unique_ptr<WriteBuffer> writeFileImpl(const String & file_name) override;
DiskPtr disk;
std::filesystem::path path;
};
}

View File

@ -16,7 +16,6 @@ using BackupEntryPtr = std::unique_ptr<IBackupEntry>;
class IBackup : public std::enable_shared_from_this<IBackup>
{
public:
IBackup() = default;
virtual ~IBackup() = default;
/// Name of the backup.
@ -24,18 +23,13 @@ public:
enum class OpenMode
{
NONE,
READ,
WRITE,
};
/// Opens the backup and start its reading or writing depending on `open_mode`.
virtual void open(OpenMode open_mode) = 0;
/// Returns whether the backup was opened for reading or writing.
virtual OpenMode getOpenMode() const = 0;
/// Closes the backup and ends its reading or writing.
virtual void close() = 0;
/// Returns the time point when this backup was created.
virtual time_t getTimestamp() const = 0;
@ -53,20 +47,24 @@ public:
/// Checks if an entry with a specified name exists.
virtual bool fileExists(const String & file_name) const = 0;
virtual bool fileExists(const std::pair<UInt64, UInt128> & size_and_checksum) const = 0;
/// Returns the size of the entry's data.
/// This function does the same as `read(file_name)->getSize()` but faster.
virtual size_t getFileSize(const String & file_name) const = 0;
virtual UInt64 getFileSize(const String & file_name) const = 0;
/// Returns the checksum of the entry's data.
/// This function does the same as `read(file_name)->getCheckum()` but faster.
virtual UInt128 getFileChecksum(const String & file_name) const = 0;
/// Finds a file by its checksum, returns nullopt if not found.
virtual std::optional<String> findFileByChecksum(const UInt128 & checksum) const = 0;
using SizeAndChecksum = std::pair<UInt64, UInt128>;
/// Returns both the size and checksum in one call.
virtual SizeAndChecksum getFileSizeAndChecksum(const String & file_name) const = 0;
/// Reads an entry from the backup.
virtual BackupEntryPtr readFile(const String & file_name) const = 0;
virtual BackupEntryPtr readFile(const SizeAndChecksum & size_and_checksum) const = 0;
/// Puts a new entry to the backup.
virtual void writeFile(const String & file_name, BackupEntryPtr entry) = 0;
@ -75,7 +73,7 @@ public:
virtual void finalizeWriting() = 0;
/// Whether it's possible to add new entries to the backup in multiple threads.
virtual bool supportsWritingInMultipleThreads() const { return true; }
virtual bool supportsWritingInMultipleThreads() const = 0;
};
using BackupPtr = std::shared_ptr<const IBackup>;

View File

@ -0,0 +1,69 @@
#pragma once
#include <Core/Types.h>
#include <optional>
namespace DB
{
/// Keeps information about files contained in a backup.
class IBackupCoordination
{
public:
struct FileInfo
{
String file_name;
UInt64 size = 0;
UInt128 checksum{0};
/// for incremental backups
UInt64 base_size = 0;
UInt128 base_checksum{0};
/// Name of the data file.
String data_file_name;
/// Suffix of an archive if the backup is stored as a series of archives.
String archive_suffix;
/// Position in the archive.
UInt64 pos_in_archive = static_cast<UInt64>(-1);
};
virtual ~IBackupCoordination() = default;
/// Adds file information.
/// If specified checksum+size are new for this IBackupContentsInfo the function sets `is_data_file_required`.
virtual void addFileInfo(const FileInfo & file_info, bool & is_data_file_required) = 0;
void addFileInfo(const FileInfo & file_info)
{
bool is_data_file_required;
addFileInfo(file_info, is_data_file_required);
}
/// Updates some fields (currently only `archive_suffix`) of a stored file's information.
virtual void updateFileInfo(const FileInfo & file_info) = 0;
virtual std::vector<FileInfo> getAllFileInfos() const = 0;
virtual Strings listFiles(const String & prefix, const String & terminator) const = 0;
using SizeAndChecksum = std::pair<UInt64, UInt128>;
virtual std::optional<FileInfo> getFileInfo(const String & file_name) const = 0;
virtual std::optional<FileInfo> getFileInfo(const SizeAndChecksum & size_and_checksum) const = 0;
virtual std::optional<SizeAndChecksum> getFileSizeAndChecksum(const String & file_name) const = 0;
/// Generates a new archive suffix, e.g. "001", "002", "003", ...
virtual String getNextArchiveSuffix() = 0;
/// Returns the list of all the archive suffixes which were generated.
virtual Strings getAllArchiveSuffixes() const = 0;
/// Removes remotely stored information.
virtual void drop() {}
};
}

View File

@ -1,5 +1,5 @@
#include <Backups/IBackupEntriesBatch.h>
#include <IO/ReadBuffer.h>
#include <IO/SeekableReadBuffer.h>
namespace DB
@ -15,7 +15,7 @@ public:
UInt64 getSize() const override { return batch->getSize(index); }
std::optional<UInt128> getChecksum() const override { return batch->getChecksum(index); }
std::unique_ptr<ReadBuffer> getReadBuffer() const override { return batch->getReadBuffer(index); }
std::unique_ptr<SeekableReadBuffer> getReadBuffer() const override { return batch->getReadBuffer(index); }
private:
const std::shared_ptr<IBackupEntriesBatch> batch;

View File

@ -17,7 +17,7 @@ public:
protected:
IBackupEntriesBatch(const Strings & entry_names_) : entry_names(entry_names_) {}
virtual std::unique_ptr<ReadBuffer> getReadBuffer(size_t index) = 0;
virtual std::unique_ptr<SeekableReadBuffer> getReadBuffer(size_t index) = 0;
virtual UInt64 getSize(size_t index) = 0;
virtual std::optional<UInt128> getChecksum(size_t) { return {}; }

View File

@ -7,7 +7,7 @@
namespace DB
{
class ReadBuffer;
class SeekableReadBuffer;
/// A backup entry represents some data which should be written to the backup or has been read from the backup.
class IBackupEntry
@ -23,7 +23,7 @@ public:
virtual std::optional<UInt128> getChecksum() const { return {}; }
/// Returns a read buffer for reading the data.
virtual std::unique_ptr<ReadBuffer> getReadBuffer() const = 0;
virtual std::unique_ptr<SeekableReadBuffer> getReadBuffer() const = 0;
};
using BackupEntryPtr = std::unique_ptr<IBackupEntry>;

View File

@ -0,0 +1,39 @@
#pragma once
#include <base/types.h>
namespace DB
{
/// Keeps information about files contained in a backup.
class IRestoreCoordination
{
public:
virtual ~IRestoreCoordination() = default;
/// Sets or gets path in the backup for a specified path in ZooKeeper.
virtual void setOrGetPathInBackupForZkPath(const String & zk_path_, String & path_in_backup_) = 0;
/// Sets that this replica is going to restore a partition in a replicated table or a table in a replicated database.
/// This function should be called to prevent other replicas from doing that in parallel.
virtual bool acquireZkPathAndName(const String & zk_path_, const String & name_) = 0;
enum Result
{
SUCCEEDED,
FAILED,
};
/// Sets the result for an acquired path and name.
virtual void setResultForZkPathAndName(const String & zk_path_, const String & name_, Result res_) = 0;
/// Waits for the result set by another replica for another replica's acquired path and name.
/// Returns false if time is out.
virtual bool getResultForZkPathAndName(const String & zk_path_, const String & name_, Result & res_, std::chrono::milliseconds timeout_) const = 0;
/// Removes remotely stored information.
virtual void drop() {}
};
}

View File

@ -0,0 +1,148 @@
#include <Backups/RestoreCoordinationDistributed.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/escapeForFileName.h>
namespace DB
{
RestoreCoordinationDistributed::RestoreCoordinationDistributed(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_)
: zookeeper_path(zookeeper_path_), get_zookeeper(get_zookeeper_)
{
createRootNodes();
}
RestoreCoordinationDistributed::~RestoreCoordinationDistributed() = default;
void RestoreCoordinationDistributed::createRootNodes()
{
auto zookeeper = get_zookeeper();
zookeeper->createAncestors(zookeeper_path);
zookeeper->createIfNotExists(zookeeper_path, "");
zookeeper->createIfNotExists(zookeeper_path + "/paths_in_backup", "");
zookeeper->createIfNotExists(zookeeper_path + "/acquired", "");
}
void RestoreCoordinationDistributed::removeAllNodes()
{
auto zookeeper = get_zookeeper();
zookeeper->removeRecursive(zookeeper_path);
}
void RestoreCoordinationDistributed::setOrGetPathInBackupForZkPath(const String & zk_path_, String & path_in_backup_)
{
{
std::lock_guard lock{mutex};
auto it = paths_in_backup_by_zk_path.find(zk_path_);
if (it != paths_in_backup_by_zk_path.end())
{
path_in_backup_ = it->second;
return;
}
}
auto zookeeper = get_zookeeper();
String combined_path = zookeeper_path + "/paths_in_backup/" + escapeForFileName(zk_path_);
auto code = zookeeper->tryCreate(combined_path, path_in_backup_, zkutil::CreateMode::Persistent);
if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS))
throw zkutil::KeeperException(code, combined_path);
if (code == Coordination::Error::ZNODEEXISTS)
path_in_backup_ = zookeeper->get(combined_path);
{
std::lock_guard lock{mutex};
paths_in_backup_by_zk_path[zk_path_] = path_in_backup_;
}
}
bool RestoreCoordinationDistributed::acquireZkPathAndName(const String & zk_path_, const String & name_)
{
std::pair<String, String> key{zk_path_, name_};
{
std::lock_guard lock{mutex};
if (acquired.contains(key))
return true;
}
auto zookeeper = get_zookeeper();
String combined_path = zookeeper_path + "/acquired/" + escapeForFileName(zk_path_) + "|" + escapeForFileName(name_);
auto code = zookeeper->tryCreate(combined_path, "", zkutil::CreateMode::Persistent);
if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS))
throw zkutil::KeeperException(code, combined_path);
if (code == Coordination::Error::ZNODEEXISTS)
return false;
{
std::lock_guard lock{mutex};
acquired.emplace(key, std::nullopt);
return true;
}
}
void RestoreCoordinationDistributed::setResultForZkPathAndName(const String & zk_path_, const String & name_, Result res_)
{
auto zookeeper = get_zookeeper();
String combined_path = zookeeper_path + "/acquired/" + escapeForFileName(zk_path_) + "|" + escapeForFileName(name_);
zookeeper->set(combined_path, (res_ == Result::SUCCEEDED) ? "1" : "0");
{
std::lock_guard lock{mutex};
acquired[std::pair{zk_path_, name_}] = res_;
}
}
bool RestoreCoordinationDistributed::getResultForZkPathAndName(const String & zk_path_, const String & name_, Result & res_, std::chrono::milliseconds timeout_) const
{
{
std::lock_guard lock{mutex};
auto value = acquired[std::pair{zk_path_, name_}];
if (value)
{
res_ = *value;
return true;
}
}
auto zookeeper = get_zookeeper();
String combined_path = zookeeper_path + "/acquired/" + escapeForFileName(zk_path_) + "|" + escapeForFileName(name_);
std::atomic<bool> changed = false;
std::condition_variable changed_condvar;
const auto watch = [&changed, &changed_condvar, zk_path_, name_](const Coordination::WatchResponse &)
{
changed = true;
changed_condvar.notify_one();
};
String res_str = zookeeper->getWatch(combined_path, nullptr, watch);
if (res_str.empty())
{
std::mutex dummy_mutex;
std::unique_lock lock{dummy_mutex};
changed_condvar.wait_for(lock, timeout_, [&changed] { return changed.load(); });
res_str = zookeeper->get(combined_path);
}
if (res_str.empty())
return false;
res_ = (res_str == "1") ? Result::SUCCEEDED : Result::FAILED;
{
std::lock_guard lock{mutex};
acquired[std::pair{zk_path_, name_}] = res_;
}
return true;
}
void RestoreCoordinationDistributed::drop()
{
removeAllNodes();
}
}

View File

@ -0,0 +1,38 @@
#pragma once
#include <Backups/IRestoreCoordination.h>
#include <Common/ZooKeeper/Common.h>
#include <map>
#include <mutex>
namespace DB
{
/// Stores restore temporary information in Zookeeper, used to perform RESTORE ON CLUSTER.
class RestoreCoordinationDistributed : public IRestoreCoordination
{
public:
RestoreCoordinationDistributed(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_);
~RestoreCoordinationDistributed() override;
void setOrGetPathInBackupForZkPath(const String & zk_path_, String & path_in_backup_) override;
bool acquireZkPathAndName(const String & zk_path_, const String & name_) override;
void setResultForZkPathAndName(const String & zk_path_, const String & name_, Result res_) override;
bool getResultForZkPathAndName(const String & zk_path_, const String & name_, Result & res_, std::chrono::milliseconds timeout_) const override;
void drop() override;
private:
void createRootNodes();
void removeAllNodes();
const String zookeeper_path;
const zkutil::GetZooKeeper get_zookeeper;
mutable std::mutex mutex;
mutable std::map<std::pair<String, String>, std::optional<Result>> acquired;
std::unordered_map<String, String> paths_in_backup_by_zk_path;
};
}

View File

@ -0,0 +1,73 @@
#include <Backups/RestoreCoordinationLocal.h>
#include <Common/Exception.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
RestoreCoordinationLocal::RestoreCoordinationLocal() = default;
RestoreCoordinationLocal::~RestoreCoordinationLocal() = default;
void RestoreCoordinationLocal::setOrGetPathInBackupForZkPath(const String & zk_path_, String & path_in_backup_)
{
std::lock_guard lock{mutex};
auto [it, inserted] = paths_in_backup_by_zk_path.try_emplace(zk_path_, path_in_backup_);
if (!inserted)
path_in_backup_ = it->second;
}
bool RestoreCoordinationLocal::acquireZkPathAndName(const String & path_, const String & name_)
{
std::lock_guard lock{mutex};
acquired.emplace(std::pair{path_, name_}, std::nullopt);
return true;
}
void RestoreCoordinationLocal::setResultForZkPathAndName(const String & zk_path_, const String & name_, Result res_)
{
std::lock_guard lock{mutex};
getResultRef(zk_path_, name_) = res_;
result_changed.notify_all();
}
bool RestoreCoordinationLocal::getResultForZkPathAndName(const String & zk_path_, const String & name_, Result & res_, std::chrono::milliseconds timeout_) const
{
std::unique_lock lock{mutex};
auto value = getResultRef(zk_path_, name_);
if (value)
{
res_ = *value;
return true;
}
bool waited = result_changed.wait_for(lock, timeout_, [this, zk_path_, name_] { return getResultRef(zk_path_, name_).has_value(); });
if (!waited)
return false;
res_ = *getResultRef(zk_path_, name_);
return true;
}
std::optional<IRestoreCoordination::Result> & RestoreCoordinationLocal::getResultRef(const String & zk_path_, const String & name_)
{
auto it = acquired.find(std::pair{zk_path_, name_});
if (it == acquired.end())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Path ({}, {}) is not acquired", zk_path_, name_);
return it->second;
}
const std::optional<IRestoreCoordination::Result> & RestoreCoordinationLocal::getResultRef(const String & zk_path_, const String & name_) const
{
auto it = acquired.find(std::pair{zk_path_, name_});
if (it == acquired.end())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Path ({}, {}) is not acquired", zk_path_, name_);
return it->second;
}
}

View File

@ -0,0 +1,34 @@
#pragma once
#include <Backups/IRestoreCoordination.h>
#include <map>
#include <mutex>
#include <unordered_map>
namespace DB
{
class RestoreCoordinationLocal : public IRestoreCoordination
{
public:
RestoreCoordinationLocal();
~RestoreCoordinationLocal() override;
void setOrGetPathInBackupForZkPath(const String & zk_path_, String & path_in_backup_) override;
bool acquireZkPathAndName(const String & zk_path_, const String & name_) override;
void setResultForZkPathAndName(const String & zk_path_, const String & name_, Result res_) override;
bool getResultForZkPathAndName(const String & zk_path_, const String & name_, Result & res_, std::chrono::milliseconds timeout_) const override;
private:
std::optional<Result> & getResultRef(const String & zk_path_, const String & name_);
const std::optional<Result> & getResultRef(const String & zk_path_, const String & name_) const;
mutable std::mutex mutex;
std::unordered_map<String, String> paths_in_backup_by_zk_path;
std::map<std::pair<String, String>, std::optional<Result>> acquired;
mutable std::condition_variable result_changed;
};
}

View File

@ -1,17 +1,80 @@
#include <Backups/RestoreSettings.h>
#include <Backups/BackupInfo.h>
#include <Backups/RestoreSettings.h>
#include <Common/FieldVisitorConvertToNumber.h>
#include <Core/SettingsFields.h>
#include <Parsers/ASTBackupQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <boost/algorithm/string/predicate.hpp>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_SETTING;
extern const int CANNOT_PARSE_BACKUP_SETTINGS;
extern const int LOGICAL_ERROR;
}
namespace
{
struct SettingFieldRestoreTableCreationMode
{
RestoreTableCreationMode value;
explicit SettingFieldRestoreTableCreationMode(const Field & field)
{
if (field.getType() == Field::Types::String)
{
const String & str = field.get<const String &>();
if (str == "1" || boost::iequals(str, "true"))
value = RestoreTableCreationMode::kCreate;
else if (str == "0" || boost::iequals(str, "false"))
value = RestoreTableCreationMode::kMustExist;
else if (boost::iequals(str, "if not exists"))
value = RestoreTableCreationMode::kCreateIfNotExists;
else throw Exception("Cannot parse creation mode from string '" + str + "'",
ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS);
}
else
{
if (applyVisitor(FieldVisitorConvertToNumber<bool>(), field))
value = RestoreTableCreationMode::kCreate;
else
value = RestoreTableCreationMode::kMustExist;
}
}
explicit operator Field() const
{
switch (value)
{
case RestoreTableCreationMode::kCreate: return Field{true};
case RestoreTableCreationMode::kMustExist: return Field{false};
case RestoreTableCreationMode::kCreateIfNotExists: return Field{"if not exists"};
}
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected value of enum RestoreTableCreationMode: {}", static_cast<int>(value));
}
};
using SettingFieldRestoreDatabaseCreationMode = SettingFieldRestoreTableCreationMode;
}
/// List of restore settings except base_backup_name.
#define LIST_OF_RESTORE_SETTINGS(M) \
M(String, password) \
M(Bool, structure_only) \
M(RestoreTableCreationMode, create_table) \
M(RestoreDatabaseCreationMode, create_database) \
M(Bool, allow_different_table_def) \
M(Bool, allow_different_database_def) \
M(Bool, async) \
M(UInt64, shard_num) \
M(UInt64, replica_num) \
M(UInt64, shard_num_in_backup) \
M(UInt64, replica_num_in_backup) \
M(Bool, internal) \
M(String, coordination_zk_path)
RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query)
{
RestoreSettings res;
@ -24,24 +87,35 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query)
const auto & settings = query.settings->as<const ASTSetQuery &>().changes;
for (const auto & setting : settings)
{
if (setting.name == "password")
res.password = SettingFieldString{setting.value};
else if (setting.name == "structure_only")
res.structure_only = SettingFieldBool{setting.value};
else if (setting.name == "throw_if_database_exists")
res.throw_if_database_exists = SettingFieldBool{setting.value};
else if (setting.name == "throw_if_table_exists")
res.throw_if_table_exists = SettingFieldBool{setting.value};
else if (setting.name == "throw_if_database_def_differs")
res.throw_if_database_def_differs = SettingFieldBool{setting.value};
else if (setting.name == "throw_if_table_def_differs")
res.throw_if_table_def_differs = SettingFieldBool{setting.value};
#define GET_SETTINGS_FROM_RESTORE_QUERY_HELPER(TYPE, NAME) \
if (setting.name == #NAME) \
res.NAME = SettingField##TYPE{setting.value}.value; \
else
throw Exception(ErrorCodes::UNKNOWN_SETTING, "Unknown setting {}", setting.name);
LIST_OF_RESTORE_SETTINGS(GET_SETTINGS_FROM_RESTORE_QUERY_HELPER)
throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.name);
}
}
return res;
}
void RestoreSettings::copySettingsToRestoreQuery(ASTBackupQuery & query) const
{
query.base_backup_name = base_backup_info ? base_backup_info->toAST() : nullptr;
auto query_settings = std::make_shared<ASTSetQuery>();
query_settings->is_standalone = false;
static const RestoreSettings default_settings;
#define SET_SETTINGS_IN_RESTORE_QUERY_HELPER(TYPE, NAME) \
if ((NAME) != default_settings.NAME) \
query_settings->changes.emplace_back(#NAME, static_cast<Field>(SettingField##TYPE{NAME}));
LIST_OF_RESTORE_SETTINGS(SET_SETTINGS_IN_RESTORE_QUERY_HELPER)
query.settings = query_settings;
}
}

View File

@ -12,6 +12,21 @@ struct StorageRestoreSettings
{
};
/// How the RESTORE command will handle table/database existence.
enum class RestoreTableCreationMode
{
/// RESTORE TABLE always tries to create a table and it throws an exception if the table already exists.
kCreate,
/// RESTORE TABLE never tries to create a table and it throws an exception if the table doesn't exist.
kMustExist,
/// RESTORE TABLE tries to create a table if it doesn't exist.
kCreateIfNotExists,
};
using RestoreDatabaseCreationMode = RestoreTableCreationMode;
/// Settings specified in the "SETTINGS" clause of a RESTORE query.
struct RestoreSettings : public StorageRestoreSettings
{
@ -27,21 +42,52 @@ struct RestoreSettings : public StorageRestoreSettings
/// without the data of tables.
bool structure_only = false;
/// Whether RESTORE DATABASE must throw an exception if a destination database already exists.
bool throw_if_database_exists = true;
/// How RESTORE command should work if a table to restore already exists.
RestoreTableCreationMode create_table = RestoreTableCreationMode::kCreateIfNotExists;
/// Whether RESTORE TABLE must throw an exception if a destination table already exists.
bool throw_if_table_exists = true;
/// How RESTORE command should work if a database to restore already exists.
RestoreDatabaseCreationMode create_database = RestoreDatabaseCreationMode::kCreateIfNotExists;
/// Whether RESTORE DATABASE must throw an exception if a destination database has
/// a different definition comparing with the definition read from backup.
bool throw_if_database_def_differs = true;
/// Normally RESTORE command throws an exception if a destination table exists but has a different definition
/// (i.e. create query) comparing with its definition extracted from backup.
/// Set `allow_different_table_def` to true to skip this check.
bool allow_different_table_def = false;
/// Whether RESTORE TABLE must throw an exception if a destination table has
/// a different definition comparing with the definition read from backup.
bool throw_if_table_def_differs = true;
/// Normally RESTORE command throws an exception if a destination database exists but has a different definition
/// (i.e. create query) comparing with its definition extracted from backup.
/// Set `allow_different_database_def` to true to skip this check.
bool allow_different_database_def = false;
/// Whether the RESTORE command must return immediately without waiting until the restoring has completed.
bool async = false;
/// 1-based shard index to restore from the backup. 0 means all shards.
/// Can only be used with RESTORE ON CLUSTER.
size_t shard_num = 0;
/// 1-based replica index to restore from the backup. 0 means all replicas.
/// Can only be used with RESTORE ON CLUSTER.
size_t replica_num = 0;
/// 1-based index of a shard stored in the backup to get data from.
/// By default it's 0: if the backup contains only one shard it means the index of that shard
/// else it means the same as `shard`.
size_t shard_num_in_backup = 0;
/// 1-based index of a replica stored in the backup to get data from.
/// By default it's 0: if the backup contains only one replica for the current shard it means the index of that replica
/// else it means the same as `replica`.
size_t replica_num_in_backup = 0;
/// Internal, should not be specified by user.
bool internal = false;
/// Internal, should not be specified by user.
/// Path in Zookeeper used to coordinate restoring process while executing by RESTORE ON CLUSTER.
String coordination_zk_path;
static RestoreSettings fromRestoreQuery(const ASTBackupQuery & query);
void copySettingsToRestoreQuery(ASTBackupQuery & query) const;
};
}

View File

@ -1,27 +1,32 @@
#include <Backups/RestoreUtils.h>
#include <Backups/BackupUtils.h>
#include <Backups/RestoreSettings.h>
#include <Backups/DDLCompareUtils.h>
#include <Backups/DDLRenamingVisitor.h>
#include <Backups/IBackup.h>
#include <Backups/IBackupEntry.h>
#include <Backups/IRestoreTask.h>
#include <Backups/RestoreSettings.h>
#include <Backups/RestoreCoordinationDistributed.h>
#include <Backups/formatTableNameOrTemporaryTableName.h>
#include <Common/escapeForFileName.h>
#include <Databases/IDatabase.h>
#include <Databases/DatabaseReplicated.h>
#include <IO/ReadHelpers.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Storages/IStorage.h>
#include <base/chrono_io.h>
#include <base/insertAtEnd.h>
#include <base/sleep.h>
#include <boost/range/adaptor/reversed.hpp>
#include <boost/range/algorithm_ext/erase.hpp>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -29,10 +34,131 @@ namespace ErrorCodes
{
extern const int CANNOT_RESTORE_TABLE;
extern const int CANNOT_RESTORE_DATABASE;
extern const int BACKUP_ENTRY_NOT_FOUND;
}
namespace
{
class PathsInBackup
{
public:
explicit PathsInBackup(const IBackup & backup_) : backup(backup_) {}
std::vector<size_t> getShards() const
{
std::vector<size_t> res;
for (const String & shard_index : backup.listFiles("shards/"))
res.push_back(parse<UInt64>(shard_index));
if (res.empty())
res.push_back(1);
return res;
}
std::vector<size_t> getReplicas(size_t shard_index) const
{
std::vector<size_t> res;
for (const String & replica_index : backup.listFiles(fmt::format("shards/{}/replicas/", shard_index)))
res.push_back(parse<UInt64>(replica_index));
if (res.empty())
res.push_back(1);
return res;
}
std::vector<String> getDatabases(size_t shard_index, size_t replica_index) const
{
std::vector<String> res;
insertAtEnd(res, backup.listFiles(fmt::format("shards/{}/replicas/{}/metadata/", shard_index, replica_index)));
insertAtEnd(res, backup.listFiles(fmt::format("shards/{}/metadata/", shard_index)));
insertAtEnd(res, backup.listFiles(fmt::format("metadata/")));
boost::range::remove_erase_if(
res,
[](String & str)
{
if (str.ends_with(".sql"))
{
str.resize(str.length() - strlen(".sql"));
str = unescapeForFileName(str);
return false;
}
return true;
});
std::sort(res.begin(), res.end());
res.erase(std::unique(res.begin(), res.end()), res.end());
return res;
}
std::vector<String> getTables(const String & database_name, size_t shard_index, size_t replica_index) const
{
std::vector<String> res;
String escaped_database_name = escapeForFileName(database_name);
insertAtEnd(res, backup.listFiles(fmt::format("shards/{}/replicas/{}/metadata/{}/", shard_index, replica_index, escaped_database_name)));
insertAtEnd(res, backup.listFiles(fmt::format("shards/{}/metadata/{}/", shard_index, escaped_database_name)));
insertAtEnd(res, backup.listFiles(fmt::format("metadata/{}/", escaped_database_name)));
boost::range::remove_erase_if(
res,
[](String & str)
{
if (str.ends_with(".sql"))
{
str.resize(str.length() - strlen(".sql"));
str = unescapeForFileName(str);
return false;
}
return true;
});
std::sort(res.begin(), res.end());
res.erase(std::unique(res.begin(), res.end()), res.end());
return res;
}
/// Returns the path to metadata in backup.
String getMetadataPath(const DatabaseAndTableName & table_name, size_t shard_index, size_t replica_index) const
{
String escaped_table_name = escapeForFileName(table_name.first) + "/" + escapeForFileName(table_name.second);
String path1 = fmt::format("shards/{}/replicas/{}/metadata/{}.sql", shard_index, replica_index, escaped_table_name);
if (backup.fileExists(path1))
return path1;
String path2 = fmt::format("shards/{}/metadata/{}.sql", shard_index, escaped_table_name);
if (backup.fileExists(path2))
return path2;
String path3 = fmt::format("metadata/{}.sql", escaped_table_name);
return path3;
}
String getMetadataPath(const String & database_name, size_t shard_index, size_t replica_index) const
{
String escaped_database_name = escapeForFileName(database_name);
String path1 = fmt::format("shards/{}/replicas/{}/metadata/{}.sql", shard_index, replica_index, escaped_database_name);
if (backup.fileExists(path1))
return path1;
String path2 = fmt::format("shards/{}/metadata/{}.sql", shard_index, escaped_database_name);
if (backup.fileExists(path2))
return path2;
String path3 = fmt::format("metadata/{}.sql", escaped_database_name);
return path3;
}
String getDataPath(const DatabaseAndTableName & table_name, size_t shard_index, size_t replica_index) const
{
String escaped_table_name = escapeForFileName(table_name.first) + "/" + escapeForFileName(table_name.second);
if (backup.fileExists(fmt::format("shards/{}/replicas/{}/metadata/{}.sql", shard_index, replica_index, escaped_table_name)))
return fmt::format("shards/{}/replicas/{}/data/{}/", shard_index, replica_index, escaped_table_name);
if (backup.fileExists(fmt::format("shards/{}/metadata/{}.sql", shard_index, escaped_table_name)))
return fmt::format("shards/{}/data/{}/", shard_index, escaped_table_name);
return fmt::format("data/{}/", escaped_table_name);
}
private:
const IBackup & backup;
};
using Kind = ASTBackupQuery::Kind;
using Element = ASTBackupQuery::Element;
using Elements = ASTBackupQuery::Elements;
@ -48,12 +174,10 @@ namespace
RestoreDatabaseTask(
ContextMutablePtr context_,
const ASTPtr & create_query_,
const RestoreSettingsPtr & restore_settings_,
bool ignore_if_database_def_differs_)
const RestoreSettingsPtr & restore_settings_)
: context(context_)
, create_query(typeid_cast<std::shared_ptr<ASTCreateQuery>>(create_query_))
, restore_settings(restore_settings_)
, ignore_if_database_def_differs(ignore_if_database_def_differs_)
{
}
@ -70,9 +194,12 @@ namespace
private:
void createDatabase()
{
/// We need to call clone() for `create_query` because the interpreter can decide
/// to change a passed AST a little bit.
InterpreterCreateQuery create_interpreter{create_query->clone(), context};
if (restore_settings->create_database == RestoreDatabaseCreationMode::kMustExist)
return;
auto cloned_create_query = typeid_cast<std::shared_ptr<ASTCreateQuery>>(create_query->clone());
cloned_create_query->if_not_exists = (restore_settings->create_database == RestoreDatabaseCreationMode::kCreateIfNotExists);
InterpreterCreateQuery create_interpreter{cloned_create_query, context};
create_interpreter.execute();
}
@ -92,7 +219,7 @@ namespace
void checkDatabaseCreateQuery()
{
if (ignore_if_database_def_differs || !restore_settings->throw_if_database_def_differs)
if (restore_settings->allow_different_database_def)
return;
getDatabaseCreateQuery();
@ -111,7 +238,6 @@ namespace
ContextMutablePtr context;
std::shared_ptr<ASTCreateQuery> create_query;
RestoreSettingsPtr restore_settings;
bool ignore_if_database_def_differs = false;
DatabasePtr database;
ASTPtr database_create_query;
};
@ -127,10 +253,11 @@ namespace
const ASTs & partitions_,
const BackupPtr & backup_,
const DatabaseAndTableName & table_name_in_backup_,
const RestoreSettingsPtr & restore_settings_)
const RestoreSettingsPtr & restore_settings_,
const std::shared_ptr<IRestoreCoordination> & restore_coordination_)
: context(context_), create_query(typeid_cast<std::shared_ptr<ASTCreateQuery>>(create_query_)),
partitions(partitions_), backup(backup_), table_name_in_backup(table_name_in_backup_),
restore_settings(restore_settings_)
restore_settings(restore_settings_), restore_coordination(restore_coordination_)
{
table_name = DatabaseAndTableName{create_query->getDatabase(), create_query->getTable()};
if (create_query->temporary)
@ -139,9 +266,28 @@ namespace
RestoreTasks run() override
{
createStorage();
getStorage();
checkStorageCreateQuery();
if (acquireTableCreation())
{
try
{
createStorage();
getStorage();
checkStorageCreateQuery();
setTableCreationResult(IRestoreCoordination::Result::SUCCEEDED);
}
catch (...)
{
setTableCreationResult(IRestoreCoordination::Result::FAILED);
throw;
}
}
else
{
waitForTableCreation();
getStorage();
checkStorageCreateQuery();
}
RestoreTasks tasks;
if (auto task = insertData())
tasks.push_back(std::move(task));
@ -151,11 +297,73 @@ namespace
bool isSequential() const override { return true; }
private:
bool acquireTableCreation()
{
if (restore_settings->create_table == RestoreTableCreationMode::kMustExist)
return true;
auto replicated_db
= typeid_cast<std::shared_ptr<const DatabaseReplicated>>(DatabaseCatalog::instance().getDatabase(table_name.first));
if (!replicated_db)
return true;
use_coordination_for_table_creation = true;
replicated_database_zookeeper_path = replicated_db->getZooKeeperPath();
return restore_coordination->acquireZkPathAndName(replicated_database_zookeeper_path, table_name.second);
}
void setTableCreationResult(IRestoreCoordination::Result res)
{
if (use_coordination_for_table_creation)
restore_coordination->setResultForZkPathAndName(replicated_database_zookeeper_path, table_name.second, res);
}
void waitForTableCreation()
{
if (!use_coordination_for_table_creation)
return;
IRestoreCoordination::Result res;
const auto & config = context->getConfigRef();
auto timeout = std::chrono::seconds(config.getUInt("backups.create_table_in_replicated_db_timeout", 10));
auto start_time = std::chrono::steady_clock::now();
if (!restore_coordination->getResultForZkPathAndName(replicated_database_zookeeper_path, table_name.second, res, timeout))
throw Exception(
ErrorCodes::CANNOT_RESTORE_TABLE,
"Waited too long ({}) for creating of {} on another replica",
to_string(timeout),
formatTableNameOrTemporaryTableName(table_name));
if (res == IRestoreCoordination::Result::FAILED)
throw Exception(
ErrorCodes::CANNOT_RESTORE_TABLE,
"Failed creating of {} on another replica",
formatTableNameOrTemporaryTableName(table_name));
while (std::chrono::steady_clock::now() - start_time < timeout)
{
if (DatabaseCatalog::instance().tryGetDatabaseAndTable({table_name.first, table_name.second}, context).second)
return;
sleepForMilliseconds(50);
}
throw Exception(
ErrorCodes::CANNOT_RESTORE_TABLE,
"Waited too long ({}) for creating of {} on another replica",
to_string(timeout),
formatTableNameOrTemporaryTableName(table_name));
}
void createStorage()
{
/// We need to call clone() for `create_query` because the interpreter can decide
/// to change a passed AST a little bit.
InterpreterCreateQuery create_interpreter{create_query->clone(), context};
if (restore_settings->create_table == RestoreTableCreationMode::kMustExist)
return;
auto cloned_create_query = typeid_cast<std::shared_ptr<ASTCreateQuery>>(create_query->clone());
cloned_create_query->if_not_exists = (restore_settings->create_table == RestoreTableCreationMode::kCreateIfNotExists);
InterpreterCreateQuery create_interpreter{cloned_create_query, context};
create_interpreter.setInternal(true);
create_interpreter.execute();
}
@ -178,7 +386,7 @@ namespace
void checkStorageCreateQuery()
{
if (!restore_settings->throw_if_table_def_differs)
if (restore_settings->allow_different_table_def)
return;
getStorageCreateQuery();
@ -203,7 +411,7 @@ namespace
if (restore_settings->structure_only)
return false;
data_path_in_backup = getDataPathInBackup(table_name_in_backup);
data_path_in_backup = PathsInBackup{*backup}.getDataPath(table_name_in_backup, restore_settings->shard_num_in_backup, restore_settings->replica_num_in_backup);
if (backup->listFiles(data_path_in_backup).empty())
return false;
@ -231,7 +439,8 @@ namespace
{
if (!hasData())
return {};
return storage->restoreData(context, partitions, backup, data_path_in_backup, *restore_settings);
return storage->restoreData(context, partitions, backup, data_path_in_backup, *restore_settings, restore_coordination);
}
ContextMutablePtr context;
@ -241,6 +450,9 @@ namespace
BackupPtr backup;
DatabaseAndTableName table_name_in_backup;
RestoreSettingsPtr restore_settings;
std::shared_ptr<IRestoreCoordination> restore_coordination;
bool use_coordination_for_table_creation = false;
String replicated_database_zookeeper_path;
DatabasePtr database;
StoragePtr storage;
ASTPtr storage_create_query;
@ -255,11 +467,17 @@ namespace
{
public:
RestoreTasksBuilder(ContextMutablePtr context_, const BackupPtr & backup_, const RestoreSettings & restore_settings_)
: context(context_), backup(backup_), restore_settings(restore_settings_) {}
: context(context_), backup(backup_), restore_settings(restore_settings_)
{
if (!restore_settings.coordination_zk_path.empty())
restore_coordination = std::make_shared<RestoreCoordinationDistributed>(restore_settings.coordination_zk_path, [context=context] { return context->getZooKeeper(); });
}
/// Prepares internal structures for making tasks for restoring.
void prepare(const ASTBackupQuery::Elements & elements)
{
adjustIndicesOfSourceShardAndReplicaInBackup();
String current_database = context->getCurrentDatabase();
renaming_settings.setFromBackupQuery(elements, current_database);
@ -296,30 +514,47 @@ namespace
/// Makes tasks for restoring, should be called after prepare().
RestoreTasks makeTasks() const
{
/// Check that there are not `different_create_query`. (If it's set it means error.)
for (const auto & info : databases | boost::adaptors::map_values)
{
if (info.different_create_query)
throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE,
"Cannot restore a database because two different create queries were generated for it: {} and {}",
serializeAST(*info.create_query), serializeAST(*info.different_create_query));
}
auto restore_settings_ptr = std::make_shared<const RestoreSettings>(restore_settings);
RestoreTasks res;
for (const auto & info : databases | boost::adaptors::map_values)
res.push_back(std::make_unique<RestoreDatabaseTask>(context, info.create_query, restore_settings_ptr,
/* ignore_if_database_def_differs = */ !info.is_explicit));
res.push_back(std::make_unique<RestoreDatabaseTask>(context, info.create_query, restore_settings_ptr));
/// TODO: We need to restore tables according to their dependencies.
for (const auto & info : tables | boost::adaptors::map_values)
res.push_back(std::make_unique<RestoreTableTask>(context, info.create_query, info.partitions, backup, info.name_in_backup, restore_settings_ptr));
res.push_back(std::make_unique<RestoreTableTask>(context, info.create_query, info.partitions, backup, info.name_in_backup, restore_settings_ptr, restore_coordination));
return res;
}
private:
void adjustIndicesOfSourceShardAndReplicaInBackup()
{
auto shards_in_backup = PathsInBackup{*backup}.getShards();
if (!restore_settings.shard_num_in_backup)
{
if (shards_in_backup.size() == 1)
restore_settings.shard_num_in_backup = shards_in_backup[0];
else
restore_settings.shard_num_in_backup = restore_settings.shard_num;
}
if (std::find(shards_in_backup.begin(), shards_in_backup.end(), restore_settings.shard_num_in_backup) == shards_in_backup.end())
throw Exception(ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "No shard #{} in backup", restore_settings.shard_num_in_backup);
auto replicas_in_backup = PathsInBackup{*backup}.getReplicas(restore_settings.shard_num_in_backup);
if (!restore_settings.replica_num_in_backup)
{
if (replicas_in_backup.size() == 1)
restore_settings.replica_num_in_backup = replicas_in_backup[0];
else
restore_settings.replica_num_in_backup = restore_settings.replica_num;
}
if (std::find(replicas_in_backup.begin(), replicas_in_backup.end(), restore_settings.replica_num_in_backup) == replicas_in_backup.end())
throw Exception(ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "No replica #{} in backup", restore_settings.replica_num_in_backup);
}
/// Prepares to restore a single table and probably its database's definition.
void prepareToRestoreTable(const DatabaseAndTableName & table_name_, const ASTs & partitions_)
{
@ -330,59 +565,12 @@ namespace
/// Make a create query for this table.
auto create_query = renameInCreateQuery(readCreateQueryFromBackup(table_name_));
create_query->if_not_exists = !restore_settings.throw_if_table_exists;
CreateTableInfo info;
info.create_query = create_query;
info.name_in_backup = table_name_;
info.partitions = partitions_;
tables[new_table_name] = std::move(info);
/// If it's not system or temporary database then probably we need to restore the database's definition too.
if (!isSystemOrTemporaryDatabase(new_table_name.first))
{
if (!databases.contains(new_table_name.first))
{
/// Add a create query for restoring the database if we haven't done it yet.
std::shared_ptr<ASTCreateQuery> create_db_query;
String db_name_in_backup = table_name_.first;
if (hasCreateQueryInBackup(db_name_in_backup))
{
create_db_query = renameInCreateQuery(readCreateQueryFromBackup(db_name_in_backup));
}
else
{
create_db_query = std::make_shared<ASTCreateQuery>();
db_name_in_backup.clear();
}
create_db_query->setDatabase(new_table_name.first);
create_db_query->if_not_exists = true;
CreateDatabaseInfo info_db;
info_db.create_query = create_db_query;
info_db.name_in_backup = std::move(db_name_in_backup);
info_db.is_explicit = false;
databases[new_table_name.first] = std::move(info_db);
}
else
{
/// We already have added a create query for restoring the database,
/// set `different_create_query` if it's not the same.
auto & info_db = databases[new_table_name.first];
if (!info_db.is_explicit && (info_db.name_in_backup != table_name_.first) && !info_db.different_create_query)
{
std::shared_ptr<ASTCreateQuery> create_db_query;
if (hasCreateQueryInBackup(table_name_.first))
create_db_query = renameInCreateQuery(readCreateQueryFromBackup(table_name_.first));
else
create_db_query = std::make_shared<ASTCreateQuery>();
create_db_query->setDatabase(new_table_name.first);
create_db_query->if_not_exists = true;
if (!areDatabaseDefinitionsSame(*info_db.create_query, *create_db_query))
info_db.different_create_query = create_db_query;
}
}
}
}
/// Prepares to restore a database and all tables in it.
@ -390,45 +578,39 @@ namespace
{
/// Check that we are not trying to restore the same database again.
String new_database_name = renaming_settings.getNewDatabaseName(database_name_);
if (databases.contains(new_database_name) && databases[new_database_name].is_explicit)
if (databases.contains(new_database_name))
throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Cannot restore the database {} twice", backQuoteIfNeed(new_database_name));
Strings table_metadata_filenames = backup->listFiles("metadata/" + escapeForFileName(database_name_) + "/", "/");
Strings table_names = PathsInBackup{*backup}.getTables(database_name_, restore_settings.shard_num_in_backup, restore_settings.replica_num_in_backup);
bool has_tables_in_backup = !table_names.empty();
bool has_create_query_in_backup = hasCreateQueryInBackup(database_name_);
bool throw_if_no_create_database_query = table_metadata_filenames.empty();
if (throw_if_no_create_database_query && !hasCreateQueryInBackup(database_name_))
if (!has_create_query_in_backup && !has_tables_in_backup)
throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Cannot restore the database {} because there is no such database in the backup", backQuoteIfNeed(database_name_));
/// Of course we're not going to restore the definition of the system or the temporary database.
if (!isSystemOrTemporaryDatabase(new_database_name))
{
/// Make a create query for this database.
std::shared_ptr<ASTCreateQuery> create_db_query;
String db_name_in_backup = database_name_;
if (hasCreateQueryInBackup(db_name_in_backup))
std::shared_ptr<ASTCreateQuery> create_query;
if (has_create_query_in_backup)
{
create_db_query = renameInCreateQuery(readCreateQueryFromBackup(db_name_in_backup));
create_query = renameInCreateQuery(readCreateQueryFromBackup(database_name_));
}
else
{
create_db_query = std::make_shared<ASTCreateQuery>();
create_db_query->setDatabase(database_name_);
db_name_in_backup.clear();
create_query = std::make_shared<ASTCreateQuery>();
create_query->setDatabase(database_name_);
}
create_db_query->if_not_exists = !restore_settings.throw_if_database_exists;
CreateDatabaseInfo info_db;
info_db.create_query = create_db_query;
info_db.name_in_backup = std::move(db_name_in_backup);
info_db.is_explicit = true;
databases[new_database_name] = std::move(info_db);
CreateDatabaseInfo info;
info.create_query = create_query;
databases[new_database_name] = std::move(info);
}
/// Restore tables in this database.
for (const String & table_metadata_filename : table_metadata_filenames)
for (const String & table_name : table_names)
{
String table_name = unescapeForFileName(fs::path{table_metadata_filename}.stem());
if (except_list_.contains(table_name))
continue;
prepareToRestoreTable(DatabaseAndTableName{database_name_, table_name}, ASTs{});
@ -438,10 +620,8 @@ namespace
/// Prepares to restore all the databases contained in the backup.
void prepareToRestoreAllDatabases(const std::set<String> & except_list_)
{
Strings database_metadata_filenames = backup->listFiles("metadata/", "/");
for (const String & database_metadata_filename : database_metadata_filenames)
for (const String & database_name : PathsInBackup{*backup}.getDatabases(restore_settings.shard_num_in_backup, restore_settings.replica_num_in_backup))
{
String database_name = unescapeForFileName(fs::path{database_metadata_filename}.stem());
if (except_list_.contains(database_name))
continue;
prepareToRestoreDatabase(database_name, std::set<String>{});
@ -451,7 +631,7 @@ namespace
/// Reads a create query for creating a specified table from the backup.
std::shared_ptr<ASTCreateQuery> readCreateQueryFromBackup(const DatabaseAndTableName & table_name) const
{
String create_query_path = getMetadataPathInBackup(table_name);
String create_query_path = PathsInBackup{*backup}.getMetadataPath(table_name, restore_settings.shard_num_in_backup, restore_settings.replica_num_in_backup);
if (!backup->fileExists(create_query_path))
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore the {} because there is no such table in the backup",
formatTableNameOrTemporaryTableName(table_name));
@ -466,7 +646,7 @@ namespace
/// Reads a create query for creating a specified database from the backup.
std::shared_ptr<ASTCreateQuery> readCreateQueryFromBackup(const String & database_name) const
{
String create_query_path = getMetadataPathInBackup(database_name);
String create_query_path = PathsInBackup{*backup}.getMetadataPath(database_name, restore_settings.shard_num_in_backup, restore_settings.replica_num_in_backup);
if (!backup->fileExists(create_query_path))
throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Cannot restore the database {} because there is no such database in the backup", backQuoteIfNeed(database_name));
auto read_buffer = backup->readFile(create_query_path)->getReadBuffer();
@ -480,7 +660,7 @@ namespace
/// Whether there is a create query for creating a specified database in the backup.
bool hasCreateQueryInBackup(const String & database_name) const
{
String create_query_path = getMetadataPathInBackup(database_name);
String create_query_path = PathsInBackup{*backup}.getMetadataPath(database_name, restore_settings.shard_num_in_backup, restore_settings.replica_num_in_backup);
return backup->fileExists(create_query_path);
}
@ -510,25 +690,15 @@ namespace
struct CreateDatabaseInfo
{
ASTPtr create_query;
String name_in_backup;
/// Whether the creation of this database is specified explicitly, via RESTORE DATABASE or
/// RESTORE ALL DATABASES.
/// It's false if the creation of this database is caused by creating a table contained in it.
bool is_explicit = false;
/// If this is set it means the following error:
/// it means that for implicitly created database there were two different create query
/// generated so we cannot restore the database.
ASTPtr different_create_query;
};
ContextMutablePtr context;
BackupPtr backup;
RestoreSettings restore_settings;
std::shared_ptr<IRestoreCoordination> restore_coordination;
DDLRenamingSettings renaming_settings;
std::map<String, CreateDatabaseInfo> databases;
std::map<DatabaseAndTableName, CreateTableInfo> tables;
std::map<String /* new_db_name */, CreateDatabaseInfo> databases;
std::map<DatabaseAndTableName /* new_table_name */, CreateTableInfo> tables;
};
@ -682,4 +852,16 @@ void executeRestoreTasks(RestoreTasks && restore_tasks, size_t num_threads)
need_rollback_completed_tasks = false;
}
size_t getMinCountOfReplicas(const IBackup & backup)
{
size_t min_count_of_replicas = static_cast<size_t>(-1);
for (size_t shard_index : PathsInBackup(backup).getShards())
{
size_t count_of_replicas = PathsInBackup(backup).getReplicas(shard_index).size();
min_count_of_replicas = std::min(min_count_of_replicas, count_of_replicas);
}
return min_count_of_replicas;
}
}

View File

@ -13,6 +13,7 @@ using RestoreTaskPtr = std::unique_ptr<IRestoreTask>;
using RestoreTasks = std::vector<RestoreTaskPtr>;
struct RestoreSettings;
class Context;
using ContextPtr = std::shared_ptr<const Context>;
using ContextMutablePtr = std::shared_ptr<Context>;
/// Prepares restore tasks.
@ -21,4 +22,7 @@ RestoreTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backu
/// Executes restore tasks.
void executeRestoreTasks(RestoreTasks && tasks, size_t num_threads);
/// Returns the minimal count of replicas stored in the backup.
size_t getMinCountOfReplicas(const IBackup & backup);
}

View File

@ -1,10 +1,10 @@
#include <Backups/BackupFactory.h>
#include <Backups/DirectoryBackup.h>
#include <Backups/ArchiveBackup.h>
#include <Backups/BackupIO_Disk.h>
#include <Backups/BackupIO_File.h>
#include <Backups/BackupImpl.h>
#include <Common/quoteString.h>
#include <Disks/IDisk.h>
#include <IO/Archives/hasRegisteredArchiveFileExtension.h>
#include <Interpreters/Context.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <filesystem>
@ -17,6 +17,7 @@ namespace ErrorCodes
extern const int INVALID_CONFIG_PARAMETER;
extern const int LOGICAL_ERROR;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int SUPPORT_IS_DISABLED;
}
@ -29,14 +30,14 @@ namespace
{
String key = "backups.allowed_disk";
if (!config.has(key))
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "The \"backups.allowed_disk\" configuration parameter is not set, cannot use Disk() backup engine");
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "The 'backups.allowed_disk' configuration parameter is not set, cannot use 'Disk' backup engine");
size_t counter = 0;
while (config.getString(key) != disk_name)
{
key = "backups.allowed_disk[" + std::to_string(++counter) + "]";
if (!config.has(key))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk {} is not allowed for backups, see the \"backups.allowed_disk\" configuration parameter", disk_name);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk {} is not allowed for backups, see the 'backups.allowed_disk' configuration parameter", quoteString(disk_name));
}
}
@ -49,7 +50,7 @@ namespace
bool path_ok = path.empty() || (path.is_relative() && (*path.begin() != ".."));
if (!path_ok)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path {} to backup must be inside the specified disk {}", quoteString(path.c_str()), disk_name);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path {} to backup must be inside the specified disk {}", quoteString(path.c_str()), quoteString(disk_name));
}
/// Checks that a path specified as parameters of File() is valid.
@ -62,7 +63,7 @@ namespace
String key = "backups.allowed_path";
if (!config.has(key))
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER,
"The \"backups.allowed_path\" configuration parameter is not set, cannot use File() backup engine");
"The 'backups.allowed_path' configuration parameter is not set, cannot use 'File' backup engine");
if (path.is_relative())
{
@ -86,7 +87,7 @@ namespace
key = "backups.allowed_path[" + std::to_string(++counter) + "]";
if (!config.has(key))
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Path {} is not allowed for backups, see the \"backups.allowed_path\" configuration parameter",
"Path {} is not allowed for backups, see the 'backups.allowed_path' configuration parameter",
quoteString(path.c_str()));
}
}
@ -104,7 +105,7 @@ void registerBackupEnginesFileAndDisk(BackupFactory & factory)
{
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Backup engine '{}' requires first argument to be a string",
"Backup engine '{}' requires its first argument to be a string",
engine_name);
}
@ -145,18 +146,41 @@ void registerBackupEnginesFileAndDisk(BackupFactory & factory)
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected backup engine '{}'", engine_name);
BackupImpl::ArchiveParams archive_params;
if (hasRegisteredArchiveFileExtension(path))
{
auto archive_backup = std::make_unique<ArchiveBackup>(backup_name, disk, path, params.context, params.base_backup_info);
archive_backup->setCompression(params.compression_method, params.compression_level);
archive_backup->setPassword(params.password);
return archive_backup;
if (params.is_internal_backup)
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Using archives with backups on clusters is disabled");
archive_params.archive_name = path.filename();
path = path.parent_path();
archive_params.compression_method = params.compression_method;
archive_params.compression_level = params.compression_level;
archive_params.password = params.password;
}
else
{
if (!params.password.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Password is not applicable, backup cannot be encrypted");
return std::make_unique<DirectoryBackup>(backup_name, disk, path, params.context, params.base_backup_info);
}
if (params.open_mode == IBackup::OpenMode::READ)
{
std::shared_ptr<IBackupReader> reader;
if (engine_name == "File")
reader = std::make_shared<BackupReaderFile>(path);
else
reader = std::make_shared<BackupReaderDisk>(disk, path);
return std::make_unique<BackupImpl>(backup_name, archive_params, params.base_backup_info, reader, params.context);
}
else
{
std::shared_ptr<IBackupWriter> writer;
if (engine_name == "File")
writer = std::make_shared<BackupWriterFile>(path);
else
writer = std::make_shared<BackupWriterDisk>(disk, path);
return std::make_unique<BackupImpl>(backup_name, archive_params, params.base_backup_info, writer, params.context, params.backup_uuid, params.is_internal_backup, params.coordination_zk_path);
}
};

View File

@ -0,0 +1,37 @@
#include <Backups/replaceTableUUIDWithMacroInReplicatedTableDef.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
namespace DB
{
void replaceTableUUIDWithMacroInReplicatedTableDef(ASTCreateQuery & create_query, const UUID & table_uuid)
{
if (create_query.getTable().empty() || !create_query.storage || !create_query.storage->engine || (table_uuid == UUIDHelpers::Nil))
return;
auto & engine = *(create_query.storage->engine);
if (!engine.name.starts_with("Replicated") || !engine.arguments)
return;
auto * args = typeid_cast<ASTExpressionList *>(engine.arguments.get());
size_t zookeeper_path_arg_pos = engine.name.starts_with("ReplicatedGraphite") ? 1 : 0;
if (!args || (args->children.size() <= zookeeper_path_arg_pos))
return;
auto * zookeeper_path_arg = typeid_cast<ASTLiteral *>(args->children[zookeeper_path_arg_pos].get());
if (!zookeeper_path_arg || (zookeeper_path_arg->value.getType() != Field::Types::String))
return;
String & zookeeper_path = zookeeper_path_arg->value.get<String>();
String table_uuid_str = toString(table_uuid);
if (size_t uuid_pos = zookeeper_path.find(table_uuid_str); uuid_pos != String::npos)
zookeeper_path.replace(uuid_pos, table_uuid_str.size(), "{uuid}");
}
}

View File

@ -0,0 +1,14 @@
#pragma once
#include <Core/UUID.h>
namespace DB
{
class ASTCreateQuery;
/// While making a replicated table it replaces "{uuid}" in zookeeper path with the real table UUID.
/// This function reverts this replacement.d
void replaceTableUUIDWithMacroInReplicatedTableDef(ASTCreateQuery & create_query, const UUID & table_uuid);
}

View File

@ -765,21 +765,9 @@ void ClientBase::receiveResult(ASTPtr parsed_query)
/// to avoid losing sync.
if (!cancelled)
{
auto cancel_query = [&] {
connection->sendCancel();
if (is_interactive)
{
progress_indication.clearProgressOutput();
std::cout << "Cancelling query." << std::endl;
}
cancelled = true;
};
/// handler received sigint
if (QueryInterruptHandler::cancelled())
{
cancel_query();
cancelQuery();
}
else
{
@ -790,7 +778,7 @@ void ClientBase::receiveResult(ASTPtr parsed_query)
<< " Waited for " << static_cast<size_t>(elapsed) << " seconds,"
<< " timeout is " << receive_timeout.totalSeconds() << " seconds." << std::endl;
cancel_query();
cancelQuery();
}
}
}
@ -1066,6 +1054,9 @@ void ClientBase::processInsertQuery(const String & query_to_execute, ASTPtr pars
return;
}
QueryInterruptHandler::start();
SCOPE_EXIT({ QueryInterruptHandler::stop(); });
connection->sendQuery(
connection_parameters.timeouts,
query,
@ -1242,6 +1233,13 @@ try
Block block;
while (executor.pull(block))
{
if (!cancelled && QueryInterruptHandler::cancelled())
{
cancelQuery();
executor.cancel();
return;
}
/// Check if server send Log packet
receiveLogs(parsed_query);
@ -1346,6 +1344,17 @@ bool ClientBase::receiveEndOfQuery()
}
}
void ClientBase::cancelQuery()
{
connection->sendCancel();
if (is_interactive)
{
progress_indication.clearProgressOutput();
std::cout << "Cancelling query." << std::endl;
}
cancelled = true;
}
void ClientBase::processParsedSingleQuery(const String & full_query, const String & query_to_execute,
ASTPtr parsed_query, std::optional<bool> echo_query_, bool report_error)

View File

@ -112,6 +112,7 @@ private:
void receiveLogs(ASTPtr parsed_query);
bool receiveSampleBlock(Block & out, ColumnsDescription & columns_description, ASTPtr parsed_query);
bool receiveEndOfQuery();
void cancelQuery();
void onProgress(const Progress & value);
void onData(Block & block, ASTPtr parsed_query);

View File

@ -621,6 +621,7 @@
M(650, SERIALIZATION_ERROR) \
M(651, CAPN_PROTO_BAD_TYPE) \
M(652, ONLY_NULLS_WHILE_READING_SCHEMA) \
M(653, CANNOT_PARSE_BACKUP_SETTINGS) \
\
M(999, KEEPER_EXCEPTION) \
M(1000, POCO_EXCEPTION) \

View File

@ -37,7 +37,7 @@ void CoordinationSettings::loadFromConfig(const String & config_elem, const Poco
}
const String KeeperConfigurationAndSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro";
const String KeeperConfigurationAndSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr";
KeeperConfigurationAndSettings::KeeperConfigurationAndSettings()
: server_id(NOT_EXIST)

View File

@ -44,11 +44,6 @@ int32_t IFourLetterCommand::toCode(const String & name)
return __builtin_bswap32(res);
}
bool IFourLetterCommand::serverIsActive() const
{
return keeper_dispatcher.hasLeader();
}
IFourLetterCommand::~IFourLetterCommand() = default;
FourLetterCommandFactory & FourLetterCommandFactory::instance()
@ -134,6 +129,9 @@ void FourLetterCommandFactory::registerCommands(KeeperDispatcher & keeper_dispat
FourLetterCommandPtr watch_command = std::make_shared<WatchCommand>(keeper_dispatcher);
factory.registerCommand(watch_command);
FourLetterCommandPtr recovery_command = std::make_shared<RecoveryCommand>(keeper_dispatcher);
factory.registerCommand(recovery_command);
factory.initializeAllowList(keeper_dispatcher);
factory.setInitialize(true);
}
@ -209,14 +207,14 @@ constexpr auto * SERVER_NOT_ACTIVE_MSG = "This instance is not currently serving
String MonitorCommand::run()
{
if (!keeper_dispatcher.isServerActive())
return SERVER_NOT_ACTIVE_MSG;
auto & stats = keeper_dispatcher.getKeeperConnectionStats();
Keeper4LWInfo keeper_info = keeper_dispatcher.getKeeper4LWInfo();
const auto & state_machine = keeper_dispatcher.getStateMachine();
if (!keeper_info.has_leader)
return SERVER_NOT_ACTIVE_MSG;
StringBuffer ret;
print(ret, "version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH);
@ -254,7 +252,7 @@ String MonitorCommand::run()
String StatResetCommand::run()
{
if (!serverIsActive())
if (!keeper_dispatcher.isServerActive())
return SERVER_NOT_ACTIVE_MSG;
keeper_dispatcher.resetConnectionStats();
@ -268,7 +266,7 @@ String NopCommand::run()
String ConfCommand::run()
{
if (!serverIsActive())
if (!keeper_dispatcher.isServerActive())
return SERVER_NOT_ACTIVE_MSG;
StringBuffer buf;
@ -278,7 +276,7 @@ String ConfCommand::run()
String ConsCommand::run()
{
if (!serverIsActive())
if (!keeper_dispatcher.isServerActive())
return SERVER_NOT_ACTIVE_MSG;
StringBuffer buf;
@ -288,7 +286,7 @@ String ConsCommand::run()
String RestConnStatsCommand::run()
{
if (!serverIsActive())
if (!keeper_dispatcher.isServerActive())
return SERVER_NOT_ACTIVE_MSG;
KeeperTCPHandler::resetConnsStats();
@ -297,7 +295,7 @@ String RestConnStatsCommand::run()
String ServerStatCommand::run()
{
if (!serverIsActive())
if (!keeper_dispatcher.isServerActive())
return SERVER_NOT_ACTIVE_MSG;
StringBuffer buf;
@ -332,7 +330,7 @@ String ServerStatCommand::run()
String StatCommand::run()
{
if (!serverIsActive())
if (!keeper_dispatcher.isServerActive())
return SERVER_NOT_ACTIVE_MSG;
StringBuffer buf;
@ -365,7 +363,7 @@ String StatCommand::run()
String BriefWatchCommand::run()
{
if (!serverIsActive())
if (!keeper_dispatcher.isServerActive())
return SERVER_NOT_ACTIVE_MSG;
StringBuffer buf;
@ -378,7 +376,7 @@ String BriefWatchCommand::run()
String WatchCommand::run()
{
if (!serverIsActive())
if (!keeper_dispatcher.isServerActive())
return SERVER_NOT_ACTIVE_MSG;
StringBuffer buf;
@ -389,7 +387,7 @@ String WatchCommand::run()
String WatchByPathCommand::run()
{
if (!serverIsActive())
if (!keeper_dispatcher.isServerActive())
return SERVER_NOT_ACTIVE_MSG;
StringBuffer buf;
@ -400,7 +398,7 @@ String WatchByPathCommand::run()
String DataSizeCommand::run()
{
if (!serverIsActive())
if (!keeper_dispatcher.isServerActive())
return SERVER_NOT_ACTIVE_MSG;
StringBuffer buf;
@ -411,7 +409,7 @@ String DataSizeCommand::run()
String DumpCommand::run()
{
if (!serverIsActive())
if (!keeper_dispatcher.isServerActive())
return SERVER_NOT_ACTIVE_MSG;
StringBuffer buf;
@ -459,4 +457,10 @@ String IsReadOnlyCommand::run()
return "rw";
}
String RecoveryCommand::run()
{
keeper_dispatcher.forceRecovery();
return "ok";
}
}

View File

@ -32,9 +32,6 @@ public:
static String toName(int32_t code);
static inline int32_t toCode(const String & name);
// Return true if server is running and serving requests
bool serverIsActive() const;
protected:
KeeperDispatcher & keeper_dispatcher;
};
@ -307,4 +304,15 @@ struct IsReadOnlyCommand : public IFourLetterCommand
~IsReadOnlyCommand() override = default;
};
struct RecoveryCommand : public IFourLetterCommand
{
explicit RecoveryCommand(KeeperDispatcher & keeper_dispatcher_)
: IFourLetterCommand(keeper_dispatcher_)
{
}
String name() override { return "rcvr"; }
String run() override;
~RecoveryCommand() override = default;
};
}

View File

@ -278,7 +278,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf
try
{
LOG_DEBUG(log, "Waiting server to initialize");
server->startup(configuration_and_settings->enable_ipv6);
server->startup(config, configuration_and_settings->enable_ipv6);
LOG_DEBUG(log, "Server initialized, waiting for quorum");
if (!start_async)
@ -367,6 +367,11 @@ void KeeperDispatcher::shutdown()
LOG_DEBUG(log, "Dispatcher shut down");
}
void KeeperDispatcher::forceRecovery()
{
server->forceRecovery();
}
KeeperDispatcher::~KeeperDispatcher()
{
shutdown();
@ -535,10 +540,18 @@ void KeeperDispatcher::updateConfigurationThread()
try
{
using namespace std::chrono_literals;
if (!server->checkInit())
{
LOG_INFO(log, "Server still not initialized, will not apply configuration until initialization finished");
std::this_thread::sleep_for(std::chrono::milliseconds(5000));
std::this_thread::sleep_for(5000ms);
continue;
}
if (server->isRecovering())
{
LOG_INFO(log, "Server is recovering, will not apply configuration until recovery is finished");
std::this_thread::sleep_for(5000ms);
continue;
}
@ -551,6 +564,9 @@ void KeeperDispatcher::updateConfigurationThread()
bool done = false;
while (!done)
{
if (server->isRecovering())
break;
if (shutdown_called)
return;
@ -574,6 +590,11 @@ void KeeperDispatcher::updateConfigurationThread()
}
}
bool KeeperDispatcher::isServerActive() const
{
return checkInit() && hasLeader() && !server->isRecovering();
}
void KeeperDispatcher::updateConfiguration(const Poco::Util::AbstractConfiguration & config)
{
auto diff = server->getConfigurationDiff(config);

View File

@ -109,11 +109,17 @@ public:
/// standalone_keeper -- we are standalone keeper application (not inside clickhouse server)
void initialize(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper, bool start_async);
void startServer();
bool checkInit() const
{
return server && server->checkInit();
}
/// Is server accepting requests, i.e. connected to the cluster
/// and achieved quorum
bool isServerActive() const;
/// Registered in ConfigReloader callback. Add new configuration changes to
/// update_configuration_queue. Keeper Dispatcher apply them asynchronously.
void updateConfiguration(const Poco::Util::AbstractConfiguration & config);
@ -121,6 +127,8 @@ public:
/// Shutdown internal keeper parts (server, state machine, log storage, etc)
void shutdown();
void forceRecovery();
/// Put request to ClickHouse Keeper
bool putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id);

View File

@ -1,21 +1,24 @@
#include <Coordination/KeeperServer.h>
#include <Coordination/Defines.h>
#include <Coordination/KeeperServer.h>
#include "config_core.h"
#include <Coordination/LoggerWrapper.h>
#include <chrono>
#include <filesystem>
#include <string>
#include <Coordination/KeeperStateMachine.h>
#include <Coordination/KeeperStateManager.h>
#include <Coordination/WriteBufferFromNuraftBuffer.h>
#include <Coordination/LoggerWrapper.h>
#include <Coordination/ReadBufferFromNuraftBuffer.h>
#include <Coordination/WriteBufferFromNuraftBuffer.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <chrono>
#include <Common/ZooKeeper/ZooKeeperIO.h>
#include <string>
#include <filesystem>
#include <Poco/Util/Application.h>
#include <boost/algorithm/string.hpp>
#include <libnuraft/cluster_config.hxx>
#include <libnuraft/raft_server.hxx>
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/Util/Application.h>
#include <Common/ZooKeeper/ZooKeeperIO.h>
namespace DB
{
@ -61,7 +64,6 @@ void setSSLParams(nuraft::asio_service::options & asio_opts)
}
#endif
std::string checkAndGetSuperdigest(const String & user_and_digest)
{
if (user_and_digest.empty())
@ -70,7 +72,8 @@ std::string checkAndGetSuperdigest(const String & user_and_digest)
std::vector<std::string> scheme_and_id;
boost::split(scheme_and_id, user_and_digest, [](char c) { return c == ':'; });
if (scheme_and_id.size() != 2 || scheme_and_id[0] != "super")
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Incorrect superdigest in keeper_server config. Must be 'super:base64string'");
throw Exception(
ErrorCodes::INVALID_CONFIG_PARAMETER, "Incorrect superdigest in keeper_server config. Must be 'super:base64string'");
return user_and_digest;
}
@ -79,7 +82,12 @@ int32_t getValueOrMaxInt32AndLogWarning(uint64_t value, const std::string & name
{
if (value > std::numeric_limits<int32_t>::max())
{
LOG_WARNING(log, "Got {} value for setting '{}' which is bigger than int32_t max value, lowering value to {}.", value, name, std::numeric_limits<int32_t>::max());
LOG_WARNING(
log,
"Got {} value for setting '{}' which is bigger than int32_t max value, lowering value to {}.",
value,
name,
std::numeric_limits<int32_t>::max());
return std::numeric_limits<int32_t>::max();
}
@ -96,23 +104,71 @@ KeeperServer::KeeperServer(
: server_id(configuration_and_settings_->server_id)
, coordination_settings(configuration_and_settings_->coordination_settings)
, state_machine(nuraft::cs_new<KeeperStateMachine>(
responses_queue_, snapshots_queue_,
configuration_and_settings_->snapshot_storage_path,
coordination_settings,
checkAndGetSuperdigest(configuration_and_settings_->super_digest)))
, state_manager(nuraft::cs_new<KeeperStateManager>(server_id, "keeper_server", configuration_and_settings_->log_storage_path, config, coordination_settings))
responses_queue_,
snapshots_queue_,
configuration_and_settings_->snapshot_storage_path,
coordination_settings,
checkAndGetSuperdigest(configuration_and_settings_->super_digest)))
, state_manager(nuraft::cs_new<KeeperStateManager>(
server_id, "keeper_server", configuration_and_settings_->log_storage_path, config, coordination_settings))
, log(&Poco::Logger::get("KeeperServer"))
, is_recovering(config.has("keeper_server.force_recovery") && config.getBool("keeper_server.force_recovery"))
{
if (coordination_settings->quorum_reads)
LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower.");
}
void KeeperServer::startup(bool enable_ipv6)
/**
* Tiny wrapper around nuraft::raft_server which adds some functions
* necessary for recovery, mostly connected to config manipulation.
*/
struct KeeperServer::KeeperRaftServer : public nuraft::raft_server
{
state_machine->init();
bool isClusterHealthy()
{
if (timer_from_init)
{
size_t expiry = get_current_params().heart_beat_interval_ * raft_server::raft_limits_.response_limit_;
state_manager->loadLogStore(state_machine->last_commit_index() + 1, coordination_settings->reserved_log_items);
if (timer_from_init->elapsedMilliseconds() < expiry)
return false;
timer_from_init.reset();
}
const size_t voting_members = get_num_voting_members();
const auto not_responding_peers = get_not_responding_peers();
const auto quorum_size = voting_members / 2 + 1;
const auto max_not_responding_peers = voting_members - quorum_size;
return not_responding_peers <= max_not_responding_peers;
}
// Manually set the internal config of the raft server
// This should be used only for recovery
void setConfig(const nuraft::ptr<nuraft::cluster_config> & new_config)
{
set_config(new_config);
}
// Manually reconfigure the cluster
// This should be used only for recovery
void forceReconfigure(const nuraft::ptr<nuraft::cluster_config> & new_config)
{
reconfigure(new_config);
}
using nuraft::raft_server::raft_server;
// peers are initially marked as responding because at least one cycle
// of heartbeat * response_limit (20) need to pass to be marked
// as not responding
// until that time passes we can't say that the cluster is healthy
std::optional<Stopwatch> timer_from_init = std::make_optional<Stopwatch>();
};
void KeeperServer::loadLatestConfig()
{
auto latest_snapshot_config = state_machine->getClusterConfig();
auto latest_log_store_config = state_manager->getLatestConfigFromLogStore();
@ -143,20 +199,60 @@ void KeeperServer::startup(bool enable_ipv6)
{
LOG_INFO(log, "No config in log store and snapshot, probably it's initial run. Will use config from .xml on disk");
}
}
void KeeperServer::enterRecoveryMode(nuraft::raft_params & params)
{
LOG_WARNING(
log,
"This instance is in recovery mode. Until the quorum is restored, no requests should be sent to any "
"of the cluster instances. This instance will start accepting requests only when the recovery is finished.");
auto latest_config = state_manager->load_config();
nuraft::ptr<nuraft::cluster_config> new_config = std::make_shared<nuraft::cluster_config>(0, latest_config ? latest_config->get_log_idx() : 0);
new_config->set_log_idx(state_manager->load_log_store()->next_slot());
new_config->get_servers() = last_local_config->get_servers();
state_manager->save_config(*new_config);
params.with_custom_commit_quorum_size(1);
params.with_custom_election_quorum_size(1);
}
void KeeperServer::forceRecovery()
{
// notify threads containing the lock that we want to enter recovery mode
is_recovering = true;
std::lock_guard lock{server_write_mutex};
auto params = raft_instance->get_current_params();
enterRecoveryMode(params);
raft_instance->setConfig(state_manager->load_config());
raft_instance->update_params(params);
}
void KeeperServer::launchRaftServer(bool enable_ipv6)
{
nuraft::raft_params params;
params.heart_beat_interval_ = getValueOrMaxInt32AndLogWarning(coordination_settings->heart_beat_interval_ms.totalMilliseconds(), "heart_beat_interval_ms", log);
params.election_timeout_lower_bound_ = getValueOrMaxInt32AndLogWarning(coordination_settings->election_timeout_lower_bound_ms.totalMilliseconds(), "election_timeout_lower_bound_ms", log);
params.election_timeout_upper_bound_ = getValueOrMaxInt32AndLogWarning(coordination_settings->election_timeout_upper_bound_ms.totalMilliseconds(), "election_timeout_upper_bound_ms", log);
params.heart_beat_interval_
= getValueOrMaxInt32AndLogWarning(coordination_settings->heart_beat_interval_ms.totalMilliseconds(), "heart_beat_interval_ms", log);
params.election_timeout_lower_bound_ = getValueOrMaxInt32AndLogWarning(
coordination_settings->election_timeout_lower_bound_ms.totalMilliseconds(), "election_timeout_lower_bound_ms", log);
params.election_timeout_upper_bound_ = getValueOrMaxInt32AndLogWarning(
coordination_settings->election_timeout_upper_bound_ms.totalMilliseconds(), "election_timeout_upper_bound_ms", log);
params.reserved_log_items_ = getValueOrMaxInt32AndLogWarning(coordination_settings->reserved_log_items, "reserved_log_items", log);
params.snapshot_distance_ = getValueOrMaxInt32AndLogWarning(coordination_settings->snapshot_distance, "snapshot_distance", log);
params.stale_log_gap_ = getValueOrMaxInt32AndLogWarning(coordination_settings->stale_log_gap, "stale_log_gap", log);
params.fresh_log_gap_ = getValueOrMaxInt32AndLogWarning(coordination_settings->fresh_log_gap, "fresh_log_gap", log);
params.client_req_timeout_ = getValueOrMaxInt32AndLogWarning(coordination_settings->operation_timeout_ms.totalMilliseconds(), "operation_timeout_ms", log);
params.client_req_timeout_
= getValueOrMaxInt32AndLogWarning(coordination_settings->operation_timeout_ms.totalMilliseconds(), "operation_timeout_ms", log);
params.auto_forwarding_ = coordination_settings->auto_forwarding;
params.auto_forwarding_req_timeout_ = std::max<uint64_t>(coordination_settings->operation_timeout_ms.totalMilliseconds() * 2, std::numeric_limits<int32_t>::max());
params.auto_forwarding_req_timeout_ = getValueOrMaxInt32AndLogWarning(coordination_settings->operation_timeout_ms.totalMilliseconds() * 2, "operation_timeout_ms", log);
params.max_append_size_ = getValueOrMaxInt32AndLogWarning(coordination_settings->max_requests_batch_size, "max_requests_batch_size", log);
params.auto_forwarding_req_timeout_
= std::max<uint64_t>(coordination_settings->operation_timeout_ms.totalMilliseconds() * 2, std::numeric_limits<int32_t>::max());
params.auto_forwarding_req_timeout_
= getValueOrMaxInt32AndLogWarning(coordination_settings->operation_timeout_ms.totalMilliseconds() * 2, "operation_timeout_ms", log);
params.max_append_size_
= getValueOrMaxInt32AndLogWarning(coordination_settings->max_requests_batch_size, "max_requests_batch_size", log);
params.return_method_ = nuraft::raft_params::async_handler;
@ -166,30 +262,19 @@ void KeeperServer::startup(bool enable_ipv6)
#if USE_SSL
setSSLParams(asio_opts);
#else
throw Exception{"SSL support for NuRaft is disabled because ClickHouse was built without SSL support.",
ErrorCodes::SUPPORT_IS_DISABLED};
throw Exception(
"SSL support for NuRaft is disabled because ClickHouse was built without SSL support.", ErrorCodes::SUPPORT_IS_DISABLED);
#endif
}
launchRaftServer(enable_ipv6, params, asio_opts);
if (is_recovering)
enterRecoveryMode(params);
if (!raft_instance)
throw Exception(ErrorCodes::RAFT_ERROR, "Cannot allocate RAFT instance");
}
void KeeperServer::launchRaftServer(
bool enable_ipv6,
const nuraft::raft_params & params,
const nuraft::asio_service::options & asio_opts)
{
nuraft::raft_server::init_options init_options;
init_options.skip_initial_election_timeout_ = state_manager->shouldStartAsFollower();
init_options.start_server_in_constructor_ = false;
init_options.raft_callback_ = [this] (nuraft::cb_func::Type type, nuraft::cb_func::Param * param)
{
return callbackFunc(type, param);
};
init_options.raft_callback_ = [this](nuraft::cb_func::Type type, nuraft::cb_func::Param * param) { return callbackFunc(type, param); };
nuraft::ptr<nuraft::logger> logger = nuraft::cs_new<LoggerWrapper>("RaftInstance", coordination_settings->raft_logs_level);
asio_service = nuraft::cs_new<nuraft::asio_service>(asio_opts, logger);
@ -205,14 +290,31 @@ void KeeperServer::launchRaftServer(
nuraft::ptr<nuraft::state_machine> casted_state_machine = state_machine;
/// raft_server creates unique_ptr from it
nuraft::context * ctx = new nuraft::context(
casted_state_manager, casted_state_machine,
asio_listener, logger, rpc_cli_factory, scheduler, params);
nuraft::context * ctx
= new nuraft::context(casted_state_manager, casted_state_machine, asio_listener, logger, rpc_cli_factory, scheduler, params);
raft_instance = nuraft::cs_new<nuraft::raft_server>(ctx, init_options);
raft_instance = nuraft::cs_new<KeeperRaftServer>(ctx, init_options);
raft_instance->start_server(init_options.skip_initial_election_timeout_);
asio_listener->listen(raft_instance);
nuraft::ptr<nuraft::raft_server> casted_raft_server = raft_instance;
asio_listener->listen(casted_raft_server);
if (!raft_instance)
throw Exception(ErrorCodes::RAFT_ERROR, "Cannot allocate RAFT instance");
}
void KeeperServer::startup(const Poco::Util::AbstractConfiguration & config, bool enable_ipv6)
{
state_machine->init();
state_manager->loadLogStore(state_machine->last_commit_index() + 1, coordination_settings->reserved_log_items);
loadLatestConfig();
last_local_config = state_manager->parseServersConfiguration(config, true).cluster_config;
launchRaftServer(enable_ipv6);
}
void KeeperServer::shutdownRaftServer()
@ -282,11 +384,14 @@ void KeeperServer::putLocalReadRequest(const KeeperStorage::RequestForSession &
RaftAppendResult KeeperServer::putRequestBatch(const KeeperStorage::RequestsForSessions & requests_for_sessions)
{
std::vector<nuraft::ptr<nuraft::buffer>> entries;
for (const auto & [session_id, time, request] : requests_for_sessions)
entries.push_back(getZooKeeperLogEntry(session_id, time, request));
std::lock_guard lock{server_write_mutex};
if (is_recovering)
return nullptr;
return raft_instance->append_entries(entries);
}
@ -337,6 +442,45 @@ uint64_t KeeperServer::getSyncedFollowerCount() const
nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type type, nuraft::cb_func::Param * param)
{
if (is_recovering)
{
switch (type)
{
case nuraft::cb_func::HeartBeat:
{
if (raft_instance->isClusterHealthy())
{
auto new_params = raft_instance->get_current_params();
new_params.custom_commit_quorum_size_ = 0;
new_params.custom_election_quorum_size_ = 0;
raft_instance->update_params(new_params);
LOG_INFO(log, "Recovery is done. You can continue using cluster normally.");
is_recovering = false;
}
break;
}
case nuraft::cb_func::NewConfig:
{
// Apply the manually set config when in recovery mode
// NuRaft will commit but skip the reconfigure if the current
// config is the same as the committed one
// Because we manually set the config to commit
// we need to call the reconfigure also
uint64_t log_idx = *static_cast<uint64_t *>(param->ctx);
if (log_idx == state_manager->load_config()->get_log_idx())
raft_instance->forceReconfigure(state_manager->load_config());
break;
}
case nuraft::cb_func::ProcessReq:
// we don't accept requests from our peers or clients
// while in recovery mode
return nuraft::cb_func::ReturnCode::ReturnNull;
default:
break;
}
}
if (initialized_flag)
return nuraft::cb_func::ReturnCode::Ok;
@ -346,7 +490,7 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ
if (next_index < last_commited || next_index - last_commited <= 1)
commited_store = true;
auto set_initialized = [this] ()
auto set_initialized = [this]()
{
std::unique_lock lock(initialized_mutex);
initialized_flag = true;
@ -411,17 +555,29 @@ std::vector<int64_t> KeeperServer::getDeadSessions()
ConfigUpdateActions KeeperServer::getConfigurationDiff(const Poco::Util::AbstractConfiguration & config)
{
return state_manager->getConfigurationDiff(config);
auto diff = state_manager->getConfigurationDiff(config);
if (!diff.empty())
{
std::lock_guard lock{server_write_mutex};
last_local_config = state_manager->parseServersConfiguration(config, true).cluster_config;
}
return diff;
}
void KeeperServer::applyConfigurationUpdate(const ConfigUpdateAction & task)
{
std::lock_guard lock{server_write_mutex};
if (is_recovering)
return;
size_t sleep_ms = 500;
if (task.action_type == ConfigUpdateActionType::AddServer)
{
LOG_INFO(log, "Will try to add server with id {}", task.server->get_id());
bool added = false;
for (size_t i = 0; i < coordination_settings->configuration_change_tries_count; ++i)
for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i)
{
if (raft_instance->get_srv_config(task.server->get_id()) != nullptr)
{
@ -438,12 +594,21 @@ void KeeperServer::applyConfigurationUpdate(const ConfigUpdateAction & task)
auto result = raft_instance->add_srv(*task.server);
if (!result->get_accepted())
LOG_INFO(log, "Command to add server {} was not accepted for the {} time, will sleep for {} ms and retry", task.server->get_id(), i + 1, sleep_ms * (i + 1));
LOG_INFO(
log,
"Command to add server {} was not accepted for the {} time, will sleep for {} ms and retry",
task.server->get_id(),
i + 1,
sleep_ms * (i + 1));
std::this_thread::sleep_for(std::chrono::milliseconds(sleep_ms * (i + 1)));
}
if (!added)
throw Exception(ErrorCodes::RAFT_ERROR, "Configuration change to add server (id {}) was not accepted by RAFT after all {} retries", task.server->get_id(), coordination_settings->configuration_change_tries_count);
throw Exception(
ErrorCodes::RAFT_ERROR,
"Configuration change to add server (id {}) was not accepted by RAFT after all {} retries",
task.server->get_id(),
coordination_settings->configuration_change_tries_count);
}
else if (task.action_type == ConfigUpdateActionType::RemoveServer)
{
@ -452,14 +617,16 @@ void KeeperServer::applyConfigurationUpdate(const ConfigUpdateAction & task)
bool removed = false;
if (task.server->get_id() == state_manager->server_id())
{
LOG_INFO(log, "Trying to remove leader node (ourself), so will yield leadership and some other node (new leader) will try remove us. "
"Probably you will have to run SYSTEM RELOAD CONFIG on the new leader node");
LOG_INFO(
log,
"Trying to remove leader node (ourself), so will yield leadership and some other node (new leader) will try remove us. "
"Probably you will have to run SYSTEM RELOAD CONFIG on the new leader node");
raft_instance->yield_leadership();
return;
}
for (size_t i = 0; i < coordination_settings->configuration_change_tries_count; ++i)
for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i)
{
if (raft_instance->get_srv_config(task.server->get_id()) == nullptr)
{
@ -476,12 +643,21 @@ void KeeperServer::applyConfigurationUpdate(const ConfigUpdateAction & task)
auto result = raft_instance->remove_srv(task.server->get_id());
if (!result->get_accepted())
LOG_INFO(log, "Command to remove server {} was not accepted for the {} time, will sleep for {} ms and retry", task.server->get_id(), i + 1, sleep_ms * (i + 1));
LOG_INFO(
log,
"Command to remove server {} was not accepted for the {} time, will sleep for {} ms and retry",
task.server->get_id(),
i + 1,
sleep_ms * (i + 1));
std::this_thread::sleep_for(std::chrono::milliseconds(sleep_ms * (i + 1)));
}
if (!removed)
throw Exception(ErrorCodes::RAFT_ERROR, "Configuration change to remove server (id {}) was not accepted by RAFT after all {} retries", task.server->get_id(), coordination_settings->configuration_change_tries_count);
throw Exception(
ErrorCodes::RAFT_ERROR,
"Configuration change to remove server (id {}) was not accepted by RAFT after all {} retries",
task.server->get_id(),
coordination_settings->configuration_change_tries_count);
}
else if (task.action_type == ConfigUpdateActionType::UpdatePriority)
raft_instance->set_priority(task.server->get_id(), task.server->get_priority());
@ -492,12 +668,14 @@ void KeeperServer::applyConfigurationUpdate(const ConfigUpdateAction & task)
bool KeeperServer::waitConfigurationUpdate(const ConfigUpdateAction & task)
{
if (is_recovering)
return false;
size_t sleep_ms = 500;
if (task.action_type == ConfigUpdateActionType::AddServer)
{
LOG_INFO(log, "Will try to wait server with id {} to be added", task.server->get_id());
for (size_t i = 0; i < coordination_settings->configuration_change_tries_count; ++i)
for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i)
{
if (raft_instance->get_srv_config(task.server->get_id()) != nullptr)
{
@ -519,7 +697,7 @@ bool KeeperServer::waitConfigurationUpdate(const ConfigUpdateAction & task)
{
LOG_INFO(log, "Will try to wait remove of server with id {}", task.server->get_id());
for (size_t i = 0; i < coordination_settings->configuration_change_tries_count; ++i)
for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i)
{
if (raft_instance->get_srv_config(task.server->get_id()) == nullptr)
{

View File

@ -1,12 +1,13 @@
#pragma once
#include <libnuraft/nuraft.hxx>
#include <Coordination/InMemoryLogStore.h>
#include <Coordination/KeeperStateManager.h>
#include <Coordination/KeeperStateMachine.h>
#include <Coordination/KeeperStorage.h>
#include <Coordination/CoordinationSettings.h>
#include <base/logger_useful.h>
#include <Coordination/InMemoryLogStore.h>
#include <Coordination/KeeperStateMachine.h>
#include <Coordination/KeeperStateManager.h>
#include <Coordination/KeeperStorage.h>
#include <libnuraft/raft_params.hxx>
#include <libnuraft/raft_server.hxx>
#include <Poco/Util/AbstractConfiguration.h>
namespace DB
{
@ -24,15 +25,23 @@ private:
nuraft::ptr<KeeperStateManager> state_manager;
nuraft::ptr<nuraft::raft_server> raft_instance;
struct KeeperRaftServer;
nuraft::ptr<KeeperRaftServer> raft_instance;
nuraft::ptr<nuraft::asio_service> asio_service;
nuraft::ptr<nuraft::rpc_listener> asio_listener;
// because some actions can be applied
// when we are sure that there are no requests currently being
// processed (e.g. recovery) we do all write actions
// on raft_server under this mutex.
mutable std::mutex server_write_mutex;
std::mutex initialized_mutex;
std::atomic<bool> initialized_flag = false;
std::condition_variable initialized_cv;
std::atomic<bool> initial_batch_committed = false;
nuraft::ptr<nuraft::cluster_config> last_local_config;
Poco::Logger * log;
/// Callback func which is called by NuRaft on all internal events.
@ -41,13 +50,16 @@ private:
/// Almost copy-paste from nuraft::launcher, but with separated server init and start
/// Allows to avoid race conditions.
void launchRaftServer(
bool enable_ipv6,
const nuraft::raft_params & params,
const nuraft::asio_service::options & asio_opts);
void launchRaftServer(bool enable_ipv6);
void shutdownRaftServer();
void loadLatestConfig();
void enterRecoveryMode(nuraft::raft_params & params);
std::atomic_bool is_recovering = false;
public:
KeeperServer(
const KeeperConfigurationAndSettingsPtr & settings_,
@ -56,12 +68,14 @@ public:
SnapshotsQueue & snapshots_queue_);
/// Load state machine from the latest snapshot and load log storage. Start NuRaft with required settings.
void startup(bool enable_ipv6 = true);
void startup(const Poco::Util::AbstractConfiguration & config, bool enable_ipv6 = true);
/// Put local read request and execute in state machine directly and response into
/// responses queue
void putLocalReadRequest(const KeeperStorage::RequestForSession & request);
bool isRecovering() const { return is_recovering; }
/// Put batch of requests into Raft and get result of put. Responses will be set separately into
/// responses_queue.
RaftAppendResult putRequestBatch(const KeeperStorage::RequestsForSessions & requests);
@ -69,10 +83,9 @@ public:
/// Return set of the non-active sessions
std::vector<int64_t> getDeadSessions();
nuraft::ptr<KeeperStateMachine> getKeeperStateMachine() const
{
return state_machine;
}
nuraft::ptr<KeeperStateMachine> getKeeperStateMachine() const { return state_machine; }
void forceRecovery();
bool isLeader() const;
@ -92,10 +105,7 @@ public:
void waitInit();
/// Return true if KeeperServer initialized
bool checkInit() const
{
return initialized_flag;
}
bool checkInit() const { return initialized_flag; }
void shutdown();

View File

@ -175,7 +175,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s)
}
void KeeperStateMachine::commit_config(const uint64_t /*log_idx*/, nuraft::ptr<nuraft::cluster_config> & new_conf)
void KeeperStateMachine::commit_config(const uint64_t /* log_idx */, nuraft::ptr<nuraft::cluster_config> & new_conf)
{
std::lock_guard lock(cluster_config_lock);
auto tmp = new_conf->serialize();

View File

@ -6,6 +6,7 @@
#include <Coordination/CoordinationSettings.h>
#include <libnuraft/nuraft.hxx>
#include <Poco/Util/AbstractConfiguration.h>
#include "Coordination/KeeperStateMachine.h"
#include <Coordination/KeeperSnapshotManager.h>
namespace DB
@ -131,6 +132,7 @@ private:
nuraft::ptr<KeeperLogStore> log_store;
nuraft::ptr<nuraft::srv_state> server_state;
public:
/// Parse configuration from xml config.
KeeperConfigurationWrapper parseServersConfiguration(const Poco::Util::AbstractConfiguration & config, bool allow_without_us) const;
};

View File

@ -247,6 +247,8 @@ class IColumn;
M(Bool, fallback_to_stale_replicas_for_distributed_queries, true, "Suppose max_replica_delay_for_distributed_queries is set and all replicas for the queried table are stale. If this setting is enabled, the query will be performed anyway, otherwise the error will be reported.", 0) \
M(UInt64, preferred_max_column_in_block_size_bytes, 0, "Limit on max column size in block while reading. Helps to decrease cache misses count. Should be close to L2 cache size.", 0) \
\
M(UInt64, parts_to_delay_insert, 150, "If the destination table contains at least that many active parts in a single partition, artificially slow down insert into table.", 0) \
M(UInt64, parts_to_throw_insert, 300, "If more than this number active parts in a single partition of the destination table, throw 'Too many parts ...' exception.", 0) \
M(Bool, insert_distributed_sync, false, "If setting is enabled, insert query into distributed waits until data will be sent to all nodes in cluster.", 0) \
M(UInt64, insert_distributed_timeout, 0, "Timeout for insert query into distributed. Setting is used only with insert_distributed_sync enabled. Zero value means no timeout.", 0) \
M(Int64, distributed_ddl_task_timeout, 180, "Timeout for DDL query responses from all hosts in cluster. If a ddl request has not been performed on all hosts, a response will contain a timeout error and a request will be executed in an async mode. Negative value means infinite. Zero means async mode.", 0) \
@ -583,6 +585,7 @@ class IColumn;
M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \
M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \
M(String, insert_deduplication_token, "", "If not empty, used for duplicate detection instead of data digest", 0) \
M(Bool, count_distinct_optimization, false, "Rewrite count distinct to subquery of group by", 0) \
M(Bool, throw_on_unsupported_query_inside_transaction, true, "Throw exception if unsupported query is used inside transaction", 0) \
M(Bool, throw_if_no_data_to_insert, true, "Enables or disables empty INSERTs, enabled by default", 0) \
// End of COMMON_SETTINGS

View File

@ -462,7 +462,7 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_
}
}
BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, ContextPtr query_context)
BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, ContextPtr query_context, bool internal)
{
if (query_context->getCurrentTransaction() && query_context->getSettingsRef().throw_on_unsupported_query_inside_transaction)
@ -471,7 +471,7 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, Contex
if (is_readonly)
throw Exception(ErrorCodes::NO_ZOOKEEPER, "Database is in readonly mode, because it cannot connect to ZooKeeper");
if (query_context->getClientInfo().query_kind != ClientInfo::QueryKind::INITIAL_QUERY)
if (!internal && (query_context->getClientInfo().query_kind != ClientInfo::QueryKind::INITIAL_QUERY))
throw Exception(ErrorCodes::INCORRECT_QUERY, "It's not initial query. ON CLUSTER is not allowed for Replicated database.");
checkQueryValid(query, query_context);

View File

@ -44,7 +44,7 @@ public:
/// Try to execute DLL query on current host as initial query. If query is succeed,
/// then it will be executed on all replicas.
BlockIO tryEnqueueReplicatedDDL(const ASTPtr & query, ContextPtr query_context);
BlockIO tryEnqueueReplicatedDDL(const ASTPtr & query, ContextPtr query_context, bool internal = false);
bool hasReplicationThread() const override { return true; }
@ -55,6 +55,8 @@ public:
String getFullReplicaName() const;
static std::pair<String, String> parseFullReplicaName(const String & name);
const String & getZooKeeperPath() const { return zookeeper_path; }
/// Returns cluster consisting of database replicas
ClusterPtr getCluster() const;

View File

@ -169,9 +169,13 @@ Pipe ClickHouseDictionarySource::createStreamForQuery(const String & query, std:
/// Sample block should not contain first row default values
auto empty_sample_block = sample_block.cloneEmpty();
auto context_copy = Context::createCopy(context);
context_copy->makeQueryContext();
if (configuration.is_local)
{
builder.init(executeQuery(query, context, true).pipeline);
builder.init(executeQuery(query, context_copy, true).pipeline);
auto converting = ActionsDAG::makeConvertingActions(
builder.getHeader().getColumnsWithTypeAndName(),
empty_sample_block.getColumnsWithTypeAndName(),
@ -185,7 +189,7 @@ Pipe ClickHouseDictionarySource::createStreamForQuery(const String & query, std:
else
{
builder.init(Pipe(std::make_shared<RemoteSource>(
std::make_shared<RemoteQueryExecutor>(pool, query, empty_sample_block, context), false, false)));
std::make_shared<RemoteQueryExecutor>(pool, query, empty_sample_block, context_copy), false, false)));
}
if (result_size_hint)

View File

@ -69,6 +69,10 @@ String AsynchronousReadIndirectBufferFromRemoteFS::getInfoForLog()
return impl->getInfoForLog();
}
std::optional<size_t> AsynchronousReadIndirectBufferFromRemoteFS::getFileSize()
{
return impl->getFileSize();
}
bool AsynchronousReadIndirectBufferFromRemoteFS::hasPendingDataToRead()
{

View File

@ -27,7 +27,7 @@ struct ReadSettings;
*
* We pass either `memory` or `prefetch_buffer` through all this chain and return it back.
*/
class AsynchronousReadIndirectBufferFromRemoteFS : public ReadBufferFromFileBase
class AsynchronousReadIndirectBufferFromRemoteFS : public ReadBufferFromFileBase, public WithFileSize
{
public:
explicit AsynchronousReadIndirectBufferFromRemoteFS(
@ -51,6 +51,8 @@ public:
String getInfoForLog() override;
std::optional<size_t> getFileSize() override;
private:
bool nextImpl() override;

View File

@ -46,7 +46,7 @@ SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const S
auto remote_file_reader_creator = [=, this]()
{
return std::make_unique<ReadBufferFromS3>(
client_ptr, bucket, remote_path, max_single_read_retries,
client_ptr, bucket, remote_path, version_id, max_single_read_retries,
settings, /* use_external_buffer */true, /* offset */ 0, read_until_position, /* restricted_seek */true);
};

View File

@ -103,6 +103,7 @@ public:
ReadBufferFromS3Gather(
std::shared_ptr<Aws::S3::S3Client> client_ptr_,
const String & bucket_,
const String & version_id_,
const std::string & common_path_prefix_,
const BlobsPathToSize & blobs_to_read_,
size_t max_single_read_retries_,
@ -110,6 +111,7 @@ public:
: ReadBufferFromRemoteFSGather(common_path_prefix_, blobs_to_read_, settings_)
, client_ptr(std::move(client_ptr_))
, bucket(bucket_)
, version_id(version_id_)
, max_single_read_retries(max_single_read_retries_)
{
}
@ -119,6 +121,7 @@ public:
private:
std::shared_ptr<Aws::S3::S3Client> client_ptr;
String bucket;
String version_id;
UInt64 max_single_read_retries;
};
#endif

View File

@ -109,6 +109,7 @@ DiskS3::DiskS3(
String name_,
String bucket_,
String s3_root_path_,
String version_id_,
DiskPtr metadata_disk_,
FileCachePtr cache_,
ContextPtr context_,
@ -116,6 +117,7 @@ DiskS3::DiskS3(
GetDiskSettings settings_getter_)
: IDiskRemote(name_, s3_root_path_, metadata_disk_, std::move(cache_), "DiskS3", settings_->thread_pool_size)
, bucket(std::move(bucket_))
, version_id(std::move(version_id_))
, current_settings(std::move(settings_))
, settings_getter(settings_getter_)
, context(context_)
@ -196,7 +198,7 @@ std::unique_ptr<ReadBufferFromFileBase> DiskS3::readFile(const String & path, co
}
auto s3_impl = std::make_unique<ReadBufferFromS3Gather>(
settings->client, bucket, metadata.remote_fs_root_path, metadata.remote_fs_objects,
settings->client, bucket, version_id, metadata.remote_fs_root_path, metadata.remote_fs_objects,
settings->s3_settings.max_single_read_retries, disk_read_settings);
if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)
@ -354,6 +356,7 @@ int DiskS3::readSchemaVersion(const String & source_bucket, const String & sourc
settings->client,
source_bucket,
source_path + SCHEMA_VERSION_OBJECT,
version_id,
settings->s3_settings.max_single_read_retries,
context->getReadSettings());

View File

@ -66,6 +66,7 @@ public:
String name_,
String bucket_,
String s3_root_path_,
String version_id_,
DiskPtr metadata_disk_,
FileCachePtr cache_,
ContextPtr context_,
@ -157,6 +158,8 @@ private:
const String bucket;
const String version_id;
MultiVersion<DiskS3Settings> current_settings;
/// Gets disk settings from context.
GetDiskSettings settings_getter;

View File

@ -195,6 +195,7 @@ void registerDiskS3(DiskFactory & factory)
name,
uri.bucket,
uri.key,
uri.version_id,
metadata_disk,
std::move(cache),
context,

View File

@ -18,10 +18,14 @@ TemporaryFileOnDisk::TemporaryFileOnDisk(const DiskPtr & disk_, const String & p
TemporaryFileOnDisk::~TemporaryFileOnDisk()
{
#if 1
if (disk && !filepath.empty())
disk->removeRecursive(filepath);
#endif
try
{
if (disk && !filepath.empty() && disk->exists(filepath))
disk->removeRecursive(filepath);
}
catch (...)
{
}
}
}

View File

@ -72,6 +72,7 @@ void registerOutputFormatMySQLWire(FormatFactory & factory);
void registerOutputFormatMarkdown(FormatFactory & factory);
void registerOutputFormatPostgreSQLWire(FormatFactory & factory);
void registerOutputFormatCapnProto(FormatFactory & factory);
void registerOutputFormatPrometheus(FormatFactory & factory);
/// Input only formats.
@ -181,6 +182,7 @@ void registerFormats()
registerOutputFormatMarkdown(factory);
registerOutputFormatPostgreSQLWire(factory);
registerOutputFormatCapnProto(factory);
registerOutputFormatPrometheus(factory);
registerInputFormatRegexp(factory);
registerInputFormatJSONAsString(factory);

View File

@ -17,6 +17,7 @@ public:
, in(std::move(in_)) {}
const ReadBuffer & getWrappedReadBuffer() const { return *in; }
ReadBuffer & getWrappedReadBuffer() { return *in; }
protected:
std::unique_ptr<ReadBuffer> in;

View File

@ -0,0 +1,139 @@
#include <IO/ConcatSeekableReadBuffer.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ARGUMENT_OUT_OF_BOUND;
}
ConcatSeekableReadBuffer::BufferInfo::~BufferInfo()
{
if (own_in)
delete in;
}
ConcatSeekableReadBuffer::ConcatSeekableReadBuffer(std::unique_ptr<SeekableReadBuffer> buf1, size_t size1, std::unique_ptr<SeekableReadBuffer> buf2, size_t size2) : ConcatSeekableReadBuffer()
{
appendBuffer(std::move(buf1), size1);
appendBuffer(std::move(buf2), size2);
}
ConcatSeekableReadBuffer::ConcatSeekableReadBuffer(SeekableReadBuffer & buf1, size_t size1, SeekableReadBuffer & buf2, size_t size2) : ConcatSeekableReadBuffer()
{
appendBuffer(buf1, size1);
appendBuffer(buf2, size2);
}
void ConcatSeekableReadBuffer::appendBuffer(std::unique_ptr<SeekableReadBuffer> buffer, size_t size)
{
appendBuffer(buffer.release(), true, size);
}
void ConcatSeekableReadBuffer::appendBuffer(SeekableReadBuffer & buffer, size_t size)
{
appendBuffer(&buffer, false, size);
}
void ConcatSeekableReadBuffer::appendBuffer(SeekableReadBuffer * buffer, bool own, size_t size)
{
BufferInfo info;
info.in = buffer;
info.own_in = own;
info.size = size;
if (!size)
return;
buffers.emplace_back(std::move(info));
total_size += size;
if (current == buffers.size() - 1)
{
working_buffer = buffers[current].in->buffer();
pos = buffers[current].in->position();
}
}
bool ConcatSeekableReadBuffer::nextImpl()
{
if (current < buffers.size())
{
buffers[current].in->position() = pos;
while ((current < buffers.size()) && buffers[current].in->eof())
{
current_start_pos += buffers[current++].size;
if (current < buffers.size())
buffers[current].in->seek(0, SEEK_SET);
}
}
if (current >= buffers.size())
{
current_start_pos = total_size;
set(nullptr, 0);
return false;
}
working_buffer = buffers[current].in->buffer();
pos = buffers[current].in->position();
return true;
}
off_t ConcatSeekableReadBuffer::getPosition()
{
size_t current_pos = current_start_pos;
if (current < buffers.size())
current_pos += buffers[current].in->getPosition() + offset();
return current_pos;
}
off_t ConcatSeekableReadBuffer::seek(off_t off, int whence)
{
off_t new_position;
off_t current_position = getPosition();
if (whence == SEEK_SET)
new_position = off;
else if (whence == SEEK_CUR)
new_position = current_position + off;
else
throw Exception("ConcatSeekableReadBuffer::seek expects SEEK_SET or SEEK_CUR as whence", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
if (new_position < 0)
throw Exception("SEEK_SET underflow: off = " + std::to_string(off), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
if (static_cast<UInt64>(new_position) > total_size)
throw Exception("SEEK_CUR shift out of bounds", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
if (static_cast<UInt64>(new_position) == total_size)
{
current = buffers.size();
current_start_pos = total_size;
set(nullptr, 0);
return new_position;
}
off_t change_position = new_position - current_position;
if ((working_buffer.begin() <= pos + change_position) && (pos + change_position <= working_buffer.end()))
{
/// Position is still inside the same working buffer.
pos += change_position;
assert(pos >= working_buffer.begin());
assert(pos <= working_buffer.end());
return new_position;
}
while (new_position < static_cast<off_t>(current_start_pos))
current_start_pos -= buffers[--current].size;
while (new_position >= static_cast<off_t>(current_start_pos + buffers[current].size))
current_start_pos += buffers[current++].size;
buffers[current].in->seek(new_position - current_start_pos, SEEK_SET);
working_buffer = buffers[current].in->buffer();
pos = buffers[current].in->position();
return new_position;
}
}

View File

@ -0,0 +1,46 @@
#pragma once
#include <IO/SeekableReadBuffer.h>
#include <vector>
namespace DB
{
/// Reads from the concatenation of multiple SeekableReadBuffer's
class ConcatSeekableReadBuffer : public SeekableReadBuffer, public WithFileSize
{
public:
ConcatSeekableReadBuffer() : SeekableReadBuffer(nullptr, 0) { }
ConcatSeekableReadBuffer(std::unique_ptr<SeekableReadBuffer> buf1, size_t size1, std::unique_ptr<SeekableReadBuffer> buf2, size_t size2);
ConcatSeekableReadBuffer(SeekableReadBuffer & buf1, size_t size1, SeekableReadBuffer & buf2, size_t size2);
void appendBuffer(std::unique_ptr<SeekableReadBuffer> buffer, size_t size);
void appendBuffer(SeekableReadBuffer & buffer, size_t size);
off_t seek(off_t off, int whence) override;
off_t getPosition() override;
std::optional<size_t> getFileSize() override { return total_size; }
private:
bool nextImpl() override;
void appendBuffer(SeekableReadBuffer * buffer, bool own, size_t size);
struct BufferInfo
{
BufferInfo() = default;
BufferInfo(BufferInfo &&) = default;
~BufferInfo();
SeekableReadBuffer * in = nullptr;
bool own_in = false;
size_t size = 0;
};
std::vector<BufferInfo> buffers;
size_t total_size = 0;
size_t current = 0;
size_t current_start_pos = 0; /// Position of the current buffer's begin.
};
}

View File

@ -0,0 +1,131 @@
#include <IO/LimitSeekableReadBuffer.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int LIMIT_EXCEEDED;
}
bool LimitSeekableReadBuffer::nextImpl()
{
if (end_position >= static_cast<off_t>(limit))
{
/// Limit reached.
set(in->position(), 0);
return false;
}
assert(position() >= in->position());
in->position() = position();
if (!in->next())
{
/// EOF reached.
set(in->position(), 0);
return false;
}
working_buffer = in->buffer();
pos = in->position();
end_position = in->getPosition() + in->available();
if (end_position > static_cast<off_t>(limit))
{
working_buffer.resize(working_buffer.size() - end_position + limit);
end_position = limit;
}
return true;
}
off_t LimitSeekableReadBuffer::seek(off_t off, int whence)
{
off_t new_position;
off_t current_position = getPosition();
if (whence == SEEK_SET)
new_position = off;
else if (whence == SEEK_CUR)
new_position = current_position + off;
else
throw Exception("LimitSeekableReadBuffer::seek expects SEEK_SET or SEEK_CUR as whence", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
if (new_position < 0)
throw Exception("SEEK_SET underflow: off = " + std::to_string(off), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
if (static_cast<UInt64>(new_position) > limit)
throw Exception("SEEK_CUR shift out of bounds", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
off_t change_position = new_position - current_position;
if ((working_buffer.begin() <= pos + change_position) && (pos + change_position <= working_buffer.end()))
{
/// Position is still inside buffer.
pos += change_position;
assert(pos >= working_buffer.begin());
assert(pos <= working_buffer.end());
return new_position;
}
in->seek(new_position, SEEK_SET);
working_buffer = in->buffer();
pos = in->position();
end_position = in->getPosition() + in->available();
if (end_position > static_cast<off_t>(limit))
{
working_buffer.resize(working_buffer.size() - end_position + limit);
end_position = limit;
}
return new_position;
}
LimitSeekableReadBuffer::LimitSeekableReadBuffer(SeekableReadBuffer * in_, bool owns, UInt64 limit_)
: SeekableReadBuffer(in_ ? in_->position() : nullptr, 0)
, in(in_)
, owns_in(owns)
, limit(limit_)
{
assert(in);
off_t current_position = in->getPosition();
if (current_position > static_cast<off_t>(limit))
throw Exception("Limit for LimitSeekableReadBuffer exceeded", ErrorCodes::LIMIT_EXCEEDED);
working_buffer = in->buffer();
pos = in->position();
end_position = current_position + in->available();
if (end_position > static_cast<off_t>(limit))
{
working_buffer.resize(working_buffer.size() - end_position + limit);
end_position = limit;
}
}
LimitSeekableReadBuffer::LimitSeekableReadBuffer(SeekableReadBuffer & in_, UInt64 limit_)
: LimitSeekableReadBuffer(&in_, false, limit_)
{
}
LimitSeekableReadBuffer::LimitSeekableReadBuffer(std::unique_ptr<SeekableReadBuffer> in_, UInt64 limit_)
: LimitSeekableReadBuffer(in_.release(), true, limit_)
{
}
LimitSeekableReadBuffer::~LimitSeekableReadBuffer()
{
/// Update underlying buffer's position in case when limit wasn't reached.
in->position() = position();
if (owns_in)
delete in;
}
}

View File

@ -0,0 +1,33 @@
#pragma once
#include <base/types.h>
#include <IO/SeekableReadBuffer.h>
namespace DB
{
/** Allows to read from another SeekableReadBuffer no far than the specified offset.
* Note that the nested SeekableReadBuffer may read slightly more data internally to fill its buffer.
*/
class LimitSeekableReadBuffer : public SeekableReadBuffer
{
public:
LimitSeekableReadBuffer(SeekableReadBuffer & in_, UInt64 limit_);
LimitSeekableReadBuffer(std::unique_ptr<SeekableReadBuffer> in_, UInt64 limit_);
~LimitSeekableReadBuffer() override;
off_t seek(off_t off, int whence) override;
off_t getPosition() override { return end_position - available(); }
private:
SeekableReadBuffer * in;
bool owns_in;
UInt64 limit;
off_t end_position; /// Offset of the end of working_buffer.
LimitSeekableReadBuffer(SeekableReadBuffer * in_, bool owns, UInt64 limit_);
bool nextImpl() override;
};
}

View File

@ -1,3 +1,4 @@
#include <mutex>
#include <unistd.h>
#include <fcntl.h>
@ -22,7 +23,7 @@ namespace ErrorCodes
}
void OpenedFile::open(int flags)
void OpenedFile::open() const
{
ProfileEvents::increment(ProfileEvents::FileOpen);
@ -33,6 +34,13 @@ void OpenedFile::open(int flags)
errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE);
}
int OpenedFile::getFD() const
{
std::lock_guard l(mutex);
if (fd == -1)
open();
return fd;
}
std::string OpenedFile::getFileName() const
{
@ -40,22 +48,24 @@ std::string OpenedFile::getFileName() const
}
OpenedFile::OpenedFile(const std::string & file_name_, int flags)
: file_name(file_name_)
OpenedFile::OpenedFile(const std::string & file_name_, int flags_)
: file_name(file_name_), flags(flags_)
{
open(flags);
}
OpenedFile::~OpenedFile()
{
if (fd != -1)
close(); /// Exceptions will lead to std::terminate and that's Ok.
close(); /// Exceptions will lead to std::terminate and that's Ok.
}
void OpenedFile::close()
{
std::lock_guard l(mutex);
if (fd == -1)
return;
if (0 != ::close(fd))
throw Exception("Cannot close file", ErrorCodes::CANNOT_CLOSE_FILE);

View File

@ -2,6 +2,7 @@
#include <Common/CurrentMetrics.h>
#include <memory>
#include <mutex>
namespace CurrentMetrics
@ -17,22 +18,25 @@ namespace DB
class OpenedFile
{
public:
OpenedFile(const std::string & file_name_, int flags);
OpenedFile(const std::string & file_name_, int flags_);
~OpenedFile();
/// Close prematurally.
void close();
int getFD() const { return fd; }
int getFD() const;
std::string getFileName() const;
private:
std::string file_name;
int fd = -1;
int flags = 0;
mutable int fd = -1;
mutable std::mutex mutex;
CurrentMetrics::Increment metric_increment{CurrentMetrics::OpenFileForRead};
void open(int flags);
void open() const;
};
}

View File

@ -14,7 +14,7 @@ namespace ErrorCodes
}
ParallelReadBuffer::ParallelReadBuffer(std::unique_ptr<ReadBufferFactory> reader_factory_, CallbackRunner schedule_, size_t max_working_readers_)
: SeekableReadBufferWithSize(nullptr, 0)
: SeekableReadBuffer(nullptr, 0)
, max_working_readers(max_working_readers_)
, schedule(std::move(schedule_))
, reader_factory(std::move(reader_factory_))
@ -116,10 +116,10 @@ off_t ParallelReadBuffer::seek(off_t offset, int whence)
return offset;
}
std::optional<size_t> ParallelReadBuffer::getTotalSize()
std::optional<size_t> ParallelReadBuffer::getFileSize()
{
std::lock_guard lock{mutex};
return reader_factory->getTotalSize();
return reader_factory->getFileSize();
}
off_t ParallelReadBuffer::getPosition()

View File

@ -22,7 +22,7 @@ namespace DB
*
* Number of working readers limited by max_working_readers.
*/
class ParallelReadBuffer : public SeekableReadBufferWithSize
class ParallelReadBuffer : public SeekableReadBuffer
{
private:
/// Blocks until data occurred in the first reader or this reader indicate finishing
@ -68,13 +68,12 @@ private:
};
public:
class ReadBufferFactory
class ReadBufferFactory : public WithFileSize
{
public:
virtual SeekableReadBufferPtr getReader() = 0;
virtual ~ReadBufferFactory() = default;
virtual ~ReadBufferFactory() override = default;
virtual off_t seek(off_t off, int whence) = 0;
virtual std::optional<size_t> getTotalSize() = 0;
};
explicit ParallelReadBuffer(std::unique_ptr<ReadBufferFactory> reader_factory_, CallbackRunner schedule_, size_t max_working_readers);
@ -82,7 +81,7 @@ public:
~ParallelReadBuffer() override { finishAndWait(); }
off_t seek(off_t off, int whence) override;
std::optional<size_t> getTotalSize() override;
std::optional<size_t> getFileSize();
off_t getPosition() override;
const ReadBufferFactory & getReadBufferFactory() const { return *reader_factory; }

View File

@ -21,6 +21,12 @@ public:
bool nextImpl() override;
bool isWithFileSize() const { return dynamic_cast<const WithFileSize *>(impl.get()) != nullptr; }
const ReadBuffer & getWrappedReadBuffer() const { return *impl; }
ReadBuffer & getWrappedReadBuffer() { return *impl; }
protected:
std::unique_ptr<SeekableReadBuffer> impl;
String file_name;

View File

@ -40,16 +40,18 @@ ReadBufferFromS3::ReadBufferFromS3(
std::shared_ptr<Aws::S3::S3Client> client_ptr_,
const String & bucket_,
const String & key_,
const String & version_id_,
UInt64 max_single_read_retries_,
const ReadSettings & settings_,
bool use_external_buffer_,
size_t offset_,
size_t read_until_position_,
bool restricted_seek_)
: SeekableReadBufferWithSize(nullptr, 0)
: SeekableReadBuffer(nullptr, 0)
, client_ptr(std::move(client_ptr_))
, bucket(bucket_)
, key(key_)
, version_id(version_id_)
, max_single_read_retries(max_single_read_retries_)
, offset(offset_)
, read_until_position(read_until_position_)
@ -128,8 +130,15 @@ bool ReadBufferFromS3::nextImpl()
ProfileEvents::increment(ProfileEvents::S3ReadMicroseconds, watch.elapsedMicroseconds());
ProfileEvents::increment(ProfileEvents::S3ReadRequestsErrors, 1);
LOG_DEBUG(log, "Caught exception while reading S3 object. Bucket: {}, Key: {}, Offset: {}, Attempt: {}, Message: {}",
bucket, key, getPosition(), attempt, e.message());
LOG_DEBUG(
log,
"Caught exception while reading S3 object. Bucket: {}, Key: {}, Version: {}, Offset: {}, Attempt: {}, Message: {}",
bucket,
key,
version_id.empty() ? "Latest" : version_id,
getPosition(),
attempt,
e.message());
if (attempt + 1 == max_single_read_retries)
throw;
@ -208,12 +217,12 @@ off_t ReadBufferFromS3::seek(off_t offset_, int whence)
return offset;
}
std::optional<size_t> ReadBufferFromS3::getTotalSize()
std::optional<size_t> ReadBufferFromS3::getFileSize()
{
if (file_size)
return file_size;
auto object_size = S3::getObjectSize(client_ptr, bucket, key, false);
auto object_size = S3::getObjectSize(client_ptr, bucket, key, version_id, false);
if (!object_size)
{
@ -248,6 +257,10 @@ std::unique_ptr<ReadBuffer> ReadBufferFromS3::initialize()
Aws::S3::Model::GetObjectRequest req;
req.SetBucket(bucket);
req.SetKey(key);
if (!version_id.empty())
{
req.SetVersionId(version_id);
}
/**
* If remote_filesystem_read_method = 'threadpool', then for MergeTree family tables
@ -259,13 +272,26 @@ std::unique_ptr<ReadBuffer> ReadBufferFromS3::initialize()
throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, read_until_position - 1);
req.SetRange(fmt::format("bytes={}-{}", offset, read_until_position - 1));
LOG_TEST(log, "Read S3 object. Bucket: {}, Key: {}, Range: {}-{}", bucket, key, offset, read_until_position - 1);
LOG_TEST(
log,
"Read S3 object. Bucket: {}, Key: {}, Version: {}, Range: {}-{}",
bucket,
key,
version_id.empty() ? "Latest" : version_id,
offset,
read_until_position - 1);
}
else
{
if (offset)
req.SetRange(fmt::format("bytes={}-", offset));
LOG_TEST(log, "Read S3 object. Bucket: {}, Key: {}, Offset: {}", bucket, key, offset);
LOG_TEST(
log,
"Read S3 object. Bucket: {}, Key: {}, Version: {}, Offset: {}",
bucket,
key,
version_id.empty() ? "Latest" : version_id,
offset);
}
Aws::S3::Model::GetObjectOutcome outcome = client_ptr->GetObject(req);
@ -293,6 +319,7 @@ SeekableReadBufferPtr ReadBufferS3Factory::getReader()
client_ptr,
bucket,
key,
version_id,
s3_max_single_read_retries,
read_settings,
false /*use_external_buffer*/,
@ -307,7 +334,7 @@ off_t ReadBufferS3Factory::seek(off_t off, [[maybe_unused]] int whence)
return off;
}
std::optional<size_t> ReadBufferS3Factory::getTotalSize()
std::optional<size_t> ReadBufferS3Factory::getFileSize()
{
return object_size;
}

View File

@ -26,12 +26,13 @@ namespace DB
/**
* Perform S3 HTTP GET request and provide response to read.
*/
class ReadBufferFromS3 : public SeekableReadBufferWithSize, public WithFileName
class ReadBufferFromS3 : public SeekableReadBuffer, public WithFileName, public WithFileSize
{
private:
std::shared_ptr<Aws::S3::S3Client> client_ptr;
String bucket;
String key;
String version_id;
UInt64 max_single_read_retries;
/// These variables are atomic because they can be used for `logging only`
@ -50,6 +51,7 @@ public:
std::shared_ptr<Aws::S3::S3Client> client_ptr_,
const String & bucket_,
const String & key_,
const String & version_id_,
UInt64 max_single_read_retries_,
const ReadSettings & settings_,
bool use_external_buffer = false,
@ -63,7 +65,7 @@ public:
off_t getPosition() override;
std::optional<size_t> getTotalSize() override;
std::optional<size_t> getFileSize() override;
void setReadUntilPosition(size_t position) override;
@ -83,6 +85,8 @@ private:
/// There is different seek policy for disk seek and for non-disk seek
/// (non-disk seek is applied for seekable input formats: orc, arrow, parquet).
bool restricted_seek;
std::optional<size_t> file_size;
};
/// Creates separate ReadBufferFromS3 for sequence of ranges of particular object
@ -93,6 +97,7 @@ public:
std::shared_ptr<Aws::S3::S3Client> client_ptr_,
const String & bucket_,
const String & key_,
const String & version_id_,
size_t range_step_,
size_t object_size_,
UInt64 s3_max_single_read_retries_,
@ -100,6 +105,7 @@ public:
: client_ptr(client_ptr_)
, bucket(bucket_)
, key(key_)
, version_id(version_id_)
, read_settings(read_settings_)
, range_generator(object_size_, range_step_)
, range_step(range_step_)
@ -114,7 +120,7 @@ public:
off_t seek(off_t off, [[maybe_unused]] int whence) override;
std::optional<size_t> getTotalSize() override;
std::optional<size_t> getFileSize() override;
String getFileName() const override { return bucket + "/" + key; }
@ -122,6 +128,7 @@ private:
std::shared_ptr<Aws::S3::S3Client> client_ptr;
const String bucket;
const String key;
const String version_id;
ReadSettings read_settings;
RangeGenerator range_generator;

View File

@ -86,7 +86,7 @@ public:
namespace detail
{
template <typename UpdatableSessionPtr>
class ReadWriteBufferFromHTTPBase : public SeekableReadBufferWithSize, public WithFileName
class ReadWriteBufferFromHTTPBase : public SeekableReadBuffer, public WithFileName, public WithFileSize
{
public:
using HTTPHeaderEntry = std::tuple<std::string, std::string>;
@ -199,7 +199,7 @@ namespace detail
}
}
std::optional<size_t> getTotalSize() override
std::optional<size_t> getFileSize() override
{
if (read_range.end)
return *read_range.end - getRangeBegin();
@ -270,7 +270,7 @@ namespace detail
bool delay_initialization = false,
bool use_external_buffer_ = false,
bool http_skip_not_found_url_ = false)
: SeekableReadBufferWithSize(nullptr, 0)
: SeekableReadBuffer(nullptr, 0)
, uri {uri_}
, method {!method_.empty() ? method_ : out_stream_callback_ ? Poco::Net::HTTPRequest::HTTP_POST : Poco::Net::HTTPRequest::HTTP_GET}
, session {session_}
@ -749,7 +749,7 @@ public:
return off;
}
std::optional<size_t> getTotalSize() override { return total_object_size; }
std::optional<size_t> getFileSize() override { return total_object_size; }
String getFileName() const override { return uri.toString(); }

View File

@ -779,13 +779,27 @@ namespace S3
static constexpr auto OBS = "OBS";
static constexpr auto OSS = "OSS";
uri = uri_;
storage_name = S3;
if (uri.getHost().empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Host is empty in S3 URI.");
/// Extract object version ID from query string.
{
version_id = "";
const String version_key = "versionId=";
const auto query_string = uri.getQuery();
auto start = query_string.rfind(version_key);
if (start != std::string::npos)
{
start += version_key.length();
auto end = query_string.find_first_of('&', start);
version_id = query_string.substr(start, end == std::string::npos ? std::string::npos : end - start);
}
}
String name;
String endpoint_authority_from_uri;
@ -842,12 +856,15 @@ namespace S3
quoteString(bucket), !uri.empty() ? " (" + uri.toString() + ")" : "");
}
size_t getObjectSize(std::shared_ptr<Aws::S3::S3Client> client_ptr, const String & bucket, const String & key, bool throw_on_error)
size_t getObjectSize(std::shared_ptr<Aws::S3::S3Client> client_ptr, const String & bucket, const String & key, const String & version_id, bool throw_on_error)
{
Aws::S3::Model::HeadObjectRequest req;
req.SetBucket(bucket);
req.SetKey(key);
if (!version_id.empty())
req.SetVersionId(version_id);
Aws::S3::Model::HeadObjectOutcome outcome = client_ptr->HeadObject(req);
if (outcome.IsSuccess())

View File

@ -66,6 +66,7 @@ struct URI
String endpoint;
String bucket;
String key;
String version_id;
String storage_name;
bool is_virtual_hosted_style;
@ -75,7 +76,7 @@ struct URI
static void validateBucket(const String & bucket, const Poco::URI & uri);
};
size_t getObjectSize(std::shared_ptr<Aws::S3::S3Client> client_ptr, const String & bucket, const String & key, bool throw_on_error = true);
size_t getObjectSize(std::shared_ptr<Aws::S3::S3Client> client_ptr, const String & bucket, const String & key, const String & version_id = {}, bool throw_on_error = true);
}

View File

@ -1,6 +1,7 @@
#pragma once
#include <IO/ReadBuffer.h>
#include <IO/WithFileSize.h>
#include <optional>
namespace DB
@ -62,20 +63,4 @@ public:
using SeekableReadBufferPtr = std::shared_ptr<SeekableReadBuffer>;
class SeekableReadBufferWithSize : public SeekableReadBuffer
{
public:
SeekableReadBufferWithSize(Position ptr, size_t size)
: SeekableReadBuffer(ptr, size) {}
SeekableReadBufferWithSize(Position ptr, size_t size, size_t offset)
: SeekableReadBuffer(ptr, size, offset) {}
/// set std::nullopt in case it is impossible to find out total size.
virtual std::optional<size_t> getTotalSize() = 0;
protected:
std::optional<size_t> file_size;
};
}

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