Compare commits

...

46 Commits

Author SHA1 Message Date
Ruihang Xia
f3a02effa7 assign partition_ranges
Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2025-04-08 21:36:29 +08:00
Ruihang Xia
52f9fc25ba Revert "feat: keep parallelize_scan unchanged"
This reverts commit 96ba00d175.
2025-04-08 21:16:09 +08:00
evenyag
214a16565a chore: update comment 2025-04-08 21:00:25 +08:00
evenyag
21790a607e feat: use smallvec 2025-04-08 20:54:34 +08:00
evenyag
b33d8c1bad fix: include build merge reader cost to scan cost 2025-04-08 20:52:19 +08:00
evenyag
916e1c2d9e fix: address compiler errors 2025-04-08 20:51:55 +08:00
evenyag
96ba00d175 feat: keep parallelize_scan unchanged 2025-04-08 20:36:48 +08:00
evenyag
7173401732 fix: use series scan in PerSeries distribution 2025-04-08 20:34:08 +08:00
evenyag
17c797a6d0 refactor: remove per series scan from SeqScan 2025-04-08 20:34:06 +08:00
evenyag
c44ba1aa69 feat: parallelize PerSeries 2025-04-08 20:26:50 +08:00
evenyag
843d33f9d0 feat: use series scan when distribution is PerSeries 2025-04-08 20:26:50 +08:00
evenyag
b74e2a7d9b feat: implement scan logic of each partition 2025-04-08 20:26:47 +08:00
evenyag
4a79c1527d chore: add to scanner enum 2025-04-08 20:24:54 +08:00
evenyag
b7a6ff9cc3 chore: basic methods for SeriesScan 2025-04-08 20:24:54 +08:00
Yingwen
609e228852 fix: get root cause of the procedure when coverting to pb (#5841) 2025-04-08 08:14:47 +00:00
Ruihang Xia
c16bae32c4 perf: evolve promql execution engine (#5691)
* use the same sort option across every prom plan

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* tweak plans

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* wip

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* fix merge compile

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* Revert "wip"

This reverts commit db58884236.

* tweak merge scan

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* handle error

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* pass distribution rule

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* reverse sort order

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* refine plans

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* more optimizations for plans

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* check logical table

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* fix tests

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* wierd tests

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* add comment

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* add test for series_divide

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* update sqlness result

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* fix scalar calculation

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* update sqlness result

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* fix: workaround join partition

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* update proto

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2025-04-08 08:12:15 +00:00
zyy17
ee4fe9d273 refactor: improve performance for Jaeger APIs (#5838)
* refactor: improve jaeger '/api/services' performance by adding the trace services table

* chore: refine some logic

* chore: compatible v0

* test: add integration test

* chore: expand default limit from 100 to 2000

* test: fix integration test

* refactor: make trace service table configurable

* refactor: use a timestamp(2100-01-01 00:00:00) as large as possible

* refactor: use '<trace_table>_services' as trace services table name
2025-04-08 02:28:06 +00:00
Yuhan Wang
6e6e335a81 feat(remote-wal): send flush request when pruning remote wal (#5825)
* feat: update minimum entry id in kvbackend

* fix: persist before delete

* chore: apply comments

* feat: add flush region in wal prune procedure

* fix: cherry-pick error

* chore: fmt

* chore: drop rx to avoid block by response

* chore: update comments

* chore: apply review comments

* test: fix unit test

* feat: add option not to flush region during wal prune

* test: fix unit test

* fix: delete at minimum replay entry id + 1

* fix: cas

* chore: add comments

* chore: apply review comments

* chore: apply review comments

* chore: fix error msg

* chore: apply review comments

* fix: idempotent cas

* refactor: use a one-way sender

* chore: better err msg

* chore: fix unit test

* chore: apply review comments

* chore: apply review comments

* chore: replace send oneway
2025-04-07 14:05:18 +00:00
Weny Xu
981d51785b fix: throw errors instead of ignoring (#5792)
* fix: throw errors instead of ignoring

* fix: fix unit tests

* refactor: remove schema version check

* fix: fix clippy

* chore: remove unused error

* refactor: remove schema version check

* feat: handle mutliple results

* feat: introduce consistency guard

* fix: release consistency guard on datanode operation completion

* test: add tests

* chore: remove schema version

* refactor: rename

* test: add more tests

* chore: print all error

* tests: query table after alteration

* log ignored request

* refine fuzz test

* chore: fix clippy and log mailbox message

* chore: close prepared statement after execution

* chore: add comment

* chore: remove log

* chore: rename to `ConsistencyPoison`

* chore: remove unused error

* fix: fix unit tests

* chore: apply suggestions from CR
2025-04-07 13:51:00 +00:00
Weny Xu
cf1eda28aa feat: add region_id to CountdownTaskHandlerExt (#5834) 2025-04-07 09:25:59 +00:00
zyy17
cf1440fc32 refactor: add time range for jager get operations API (#5791)
* refactor: add default time range for jager get operations API

* refactor: use desc order for timestamp colomn

* chore: modify http header name
2025-04-07 09:07:31 +00:00
Yingwen
21a209f7ba fix: skip replacing exprs of the DistinctOn node (#5823)
* fix: handle distinct on specially

* chore: update comment
2025-04-07 08:59:40 +00:00
Weny Xu
917510ffd0 feat: introduce poison mechanism for procedure (#5822)
* feat: introduce poison for procedure

* tests: add unit tests

* refactor: minor refactor

* fix: unit tests

* chore: fix unit tests

* chore: apply suggestions from CR

* chore: apply suggestions from CR

* chore: update comments

* chore: introduce `ProcedureStatus::Poisoned`

* chore: upgrade greptime-proto to `2be0f`

* chore: apply suggestions from CR
2025-04-07 08:25:13 +00:00
fys
7b48ef1e97 chore: remove patch.crates-io for rustls (#5832)
* chore: remove patch.crates-io for rustls

* enable default-rustls-ring feature for mysql_sync

* fix: build error

* add comment

* update comment
2025-04-07 07:51:50 +00:00
Weny Xu
ac0f9ab575 refactor: remove backoff config (#5808)
* refactor: remove backoff config

* chore: update config.md

* fix: correct backoff config

* chore: change deadline to 120s
2025-04-07 07:22:22 +00:00
Ning Sun
f2907bb009 refactor!: make pipeline a required parameter when ingesting trace (#5828)
* feat: make pipeline a required header for trace

* test: add test case without pipeline
2025-04-07 06:18:17 +00:00
Ryan Despain
1695919ee7 clear message for an awesome achievement (#5829)
Initially there was what I think was a typo. `s/archive/achieve` but then I thought some clarification might be nice on this great achievement.
2025-04-07 02:37:19 +00:00
Weny Xu
eab702cc02 feat: implement sync_region for metric engine (#5826)
* feat: implement `sync_region` for metric engine

* chore: apply suggestions from CR

* chore: upgrade proto
2025-04-03 12:46:20 +00:00
Zhenchi
dd63068df6 feat: add matches_term function (#5817)
* feat: add `matches_term` function

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

* merge & fix

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

* address comments

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

* fix & skip char after boundary mismatch

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

---------

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>
2025-04-03 09:09:41 +00:00
Yuhan Wang
f73b61e767 feat(remote-wal): add remote wal prune procedure (#5714)
* feat: add remote wal prune procedure

* feat: add retry logic and remove rollback

* chore: simplify the logic

* fix: remove REMOTE_WAL_LOCK

* fix: use in-memory kv

* perf: O(n) judgement

* chore: add single write lock

* test: add unit test

* chore: remove unused function

* chore: update comments

* chore: apply comments

* chore: apply comments
2025-04-03 08:11:51 +00:00
Yingwen
2acecd3620 feat: support REPLACE INTO statement (#5820)
* feat: support replace into

* feat: support replace into
2025-04-03 03:22:43 +00:00
Zhenchi
f797de3497 feat: add backend field to fulltext options (#5806)
* feat: add backend field to fulltext options

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

* update proto

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

* fix option conv

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

* fix display

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

* polish

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

---------

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>
2025-04-02 09:15:54 +00:00
dennis zhuang
d53afa849d fix: interval cast expression can't work in range query, #5805 (#5813)
* fix: interval cast expression can't work in range query, #5805

* fix: nested cast

* test: make vector test stable
2025-04-02 08:46:17 +00:00
discord9
3aebfc1716 test: looser condition (#5816) 2025-04-02 07:38:05 +00:00
Weny Xu
dbb79c9671 feat: introduce CollectLeaderRegionHandler (#5811)
* feat: introduce `CollectLeaderRegionHandler`

* feat: add to default handler group

* fix: correct unit test

* chore: rename
2025-04-02 04:47:00 +00:00
shuiyisong
054056fcbb refactor: remove prom store write dispatch (#5812)
* refactor: remove prom store remote write dispatch pattern

* chore: ref XIX-22
2025-04-02 04:35:28 +00:00
Zhenchi
aa486db8b7 refactor: allow bloom filter search to apply and conjunction (#5770)
* refactor: change bloom filter search from any to all match

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

* polish

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

* place back in list

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

* nit

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

---------

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>
2025-04-01 12:50:34 +00:00
Weny Xu
4ef9afd8d8 feat: introduce read preference (#5783)
* feat: introduce read preference

* feat: introduce `RegionQueryHandlerFactory`

* feat: extract ReadPreference from http header

* test: add more tests

* chore: apply suggestions from CR

* chore: apply suggestions from CR
2025-04-01 09:17:01 +00:00
shuiyisong
f9221e9e66 perf: introduce simd_json for parsing ndjson (#5794)
* perf: introduce simd_json for parsing ndjson

* fix: some tests

* fix: some tests

* fix: es test case

* chore: use `as_bytes_mut()`

* chore: remove unnecessary `to_string`

* chore: add safety comment
2025-04-01 08:17:26 +00:00
Weny Xu
6c26fe9c80 fix: correct error status code (#5802) 2025-04-01 07:34:16 +00:00
fys
33c9fb737c refactor: remove mode option in configuration files (#5809)
* refactor: remove mode option in configuration files

* chore: remove mode in configuration file

* remvoe mode field in FlownodeOptions

* add comment for test

* update config.md

* remove mode field in standalone options

* fix: ci
2025-04-01 07:14:10 +00:00
Weny Xu
68ce796771 chore: expose modules (#5810) 2025-04-01 05:33:20 +00:00
Weny Xu
d701c18150 feat: introduce CustomizedRegionLeaseRenewer (#5762)
* feat: add manifest_version to `GrantedRegion`

* chore: upgrade proto

* chore: apply review suggestions

* chore: apply suggestions from CR

* feat: introduce `CustomizedRegionLeaseRenewerRef`

* chore: upgrade to `103948`
2025-03-31 13:25:05 +00:00
Weny Xu
d3a60d8821 feat: add limit for the number of running procedures (#5793)
* refactor: remove unused `messages`

* feat: introduce running procedure num limit

* feat: update config

* chore: apply suggestions from CR

* feat: impl `status_code` for `log-store` crate
2025-03-31 06:14:21 +00:00
discord9
5d688c6565 feat(flow): time window expr (#5785)
* feat: time window expr

* chore: comments

* refactor: per review

* chore: partially per review

* chore: per review

* chore: per review use query engine's session
2025-03-31 04:46:37 +00:00
Weny Xu
41aee1f1b7 feat: implement sync_region for mito engine (#5765)
* chore: upgrade proto to `2d52b`

* feat: add `SyncRegion` to `WorkerRequest`

* feat: impl `sync_region` for `Engine` trait

* test: add tests

* chore: fmt code

* chore: upgrade proto

* chore: unify `RegionLeaderState` and `RegionFollowerState`

* chore: check immutable memtable

* chore: fix clippy

* chore: apply suggestions from CR
2025-03-31 03:53:47 +00:00
244 changed files with 9797 additions and 2117 deletions

442
Cargo.lock generated
View File

@@ -2038,6 +2038,7 @@ dependencies = [
"h3o",
"hyperloglogplus",
"jsonb",
"memchr",
"nalgebra 0.33.2",
"num",
"num-traits",
@@ -2267,6 +2268,7 @@ version = "0.14.0"
dependencies = [
"async-trait",
"common-procedure",
"snafu 0.8.5",
]
[[package]]
@@ -2344,6 +2346,13 @@ dependencies = [
"tokio-util",
]
[[package]]
name = "common-session"
version = "0.14.0"
dependencies = [
"strum 0.27.1",
]
[[package]]
name = "common-telemetry"
version = "0.14.0"
@@ -2424,9 +2433,9 @@ dependencies = [
"humantime-serde",
"num_cpus",
"rskafka",
"rustls 0.23.20",
"rustls",
"rustls-native-certs 0.7.3",
"rustls-pemfile 2.2.0",
"rustls-pemfile",
"serde",
"serde_json",
"serde_with",
@@ -2744,9 +2753,9 @@ dependencies = [
[[package]]
name = "crossbeam-queue"
version = "0.3.11"
version = "0.3.12"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "df0346b5d5e76ac2fe4e327c5fd1118d6be7c51dfb18f9b7922923f287471e35"
checksum = "0f58bbc28f91df819d0aa2a2c00cd19754769c2fad90579b3592b1c9ba7a3115"
dependencies = [
"crossbeam-utils",
]
@@ -4091,6 +4100,15 @@ dependencies = [
"miniz_oxide",
]
[[package]]
name = "float-cmp"
version = "0.10.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "b09cf3155332e944990140d967ff5eceb70df778b34f77d8075db46e4704e6d8"
dependencies = [
"num-traits",
]
[[package]]
name = "float_eq"
version = "1.0.1"
@@ -4626,9 +4644,11 @@ source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "73fea8450eea4bac3940448fb7ae50d91f034f941199fcd9d909a5a07aa455f0"
dependencies = [
"cfg-if",
"js-sys",
"libc",
"r-efi",
"wasi 0.14.2+wasi-0.2.4",
"wasm-bindgen",
]
[[package]]
@@ -4671,7 +4691,7 @@ dependencies = [
[[package]]
name = "greptime-proto"
version = "0.1.0"
source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=97e298d119fdb9499bc6ba9e03f375cfa7cdf130#97e298d119fdb9499bc6ba9e03f375cfa7cdf130"
source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=dd4a1996982534636734674db66e44464b0c0d83#dd4a1996982534636734674db66e44464b0c0d83"
dependencies = [
"prost 0.13.3",
"serde",
@@ -4750,6 +4770,16 @@ dependencies = [
"num-traits",
]
[[package]]
name = "halfbrown"
version = "0.3.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "aa2c385c6df70fd180bbb673d93039dbd2cd34e41d782600bdf6e1ca7bce39aa"
dependencies = [
"hashbrown 0.15.2",
"serde",
]
[[package]]
name = "hash32"
version = "0.3.1"
@@ -5196,27 +5226,28 @@ dependencies = [
"pin-project-lite",
"rustls-native-certs 0.7.3",
"tokio",
"tokio-rustls 0.26.0",
"tokio-rustls",
"tower-service",
]
[[package]]
name = "hyper-rustls"
version = "0.27.3"
source = "git+https://github.com/GreptimeTeam/hyper-rustls?rev=a951e03#a951e03fb914f1830e244400472814d38775118d"
version = "0.27.5"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "2d191583f3da1305256f22463b9bb0471acad48a4e534a5218b9963e9c1f59b2"
dependencies = [
"futures-util",
"http 1.1.0",
"hyper 1.4.1",
"hyper-util",
"log",
"rustls 0.23.20",
"rustls",
"rustls-native-certs 0.8.0",
"rustls-pki-types",
"tokio",
"tokio-rustls 0.26.0",
"tokio-rustls",
"tower-service",
"webpki-roots 0.26.6",
"webpki-roots",
]
[[package]]
@@ -5815,10 +5846,11 @@ dependencies = [
[[package]]
name = "js-sys"
version = "0.3.70"
version = "0.3.77"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "1868808506b929d7b0cfa8f75951347aa71bb21144b7791bae35d9bccfcfe37a"
checksum = "1cfaf33c695fc6e08064efbc1f72ec937429614f25eef83af942d0e227c3a28f"
dependencies = [
"once_cell",
"wasm-bindgen",
]
@@ -5993,8 +6025,8 @@ dependencies = [
"k8s-openapi",
"kube-core",
"pem",
"rustls 0.23.20",
"rustls-pemfile 2.2.0",
"rustls",
"rustls-pemfile",
"secrecy",
"serde",
"serde_json",
@@ -6482,7 +6514,7 @@ version = "0.11.3"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "75761162ae2b0e580d7e7c390558127e5f01b4194debd6221fd8c207fc80e3f5"
dependencies = [
"twox-hash",
"twox-hash 1.6.3",
]
[[package]]
@@ -6691,12 +6723,14 @@ dependencies = [
"itertools 0.14.0",
"lazy_static",
"local-ip-address",
"log-store",
"once_cell",
"parking_lot 0.12.3",
"prometheus",
"prost 0.13.3",
"rand 0.9.0",
"regex",
"rskafka",
"serde",
"serde_json",
"servers",
@@ -6812,18 +6846,6 @@ dependencies = [
"adler2",
]
[[package]]
name = "mio"
version = "0.8.11"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "a4a650543ca06a924e8b371db273b2756685faae30f8487da1b56505a8f78b0c"
dependencies = [
"libc",
"log",
"wasi 0.11.0+wasi-snapshot-preview1",
"windows-sys 0.48.0",
]
[[package]]
name = "mio"
version = "1.0.2"
@@ -6993,48 +7015,28 @@ checksum = "2195bf6aa996a481483b29d62a7663eed3fe39600c460e323f8ff41e90bdd89b"
[[package]]
name = "mysql"
version = "25.0.1"
version = "26.0.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "c6ad644efb545e459029b1ffa7c969d830975bd76906820913247620df10050b"
checksum = "64453aedc258ac8c720b46c8264302fad39cef6c02483f68adbad4bcd22d6fab"
dependencies = [
"bufstream",
"bytes",
"crossbeam",
"crossbeam-queue",
"flate2",
"io-enum",
"libc",
"lru",
"mysql_common 0.32.4",
"mysql_common 0.34.1",
"named_pipe",
"pem",
"percent-encoding",
"rustls 0.23.20",
"rustls-pemfile 2.2.0",
"serde",
"serde_json",
"rustls",
"rustls-pemfile",
"socket2",
"twox-hash",
"twox-hash 2.1.0",
"url",
"webpki",
"webpki-roots 0.26.6",
]
[[package]]
name = "mysql-common-derive"
version = "0.30.2"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "56b0d8a0db9bf6d2213e11f2c701cb91387b0614361625ab7b9743b41aa4938f"
dependencies = [
"darling 0.20.10",
"heck 0.4.1",
"num-bigint",
"proc-macro-crate 1.3.1",
"proc-macro-error",
"proc-macro2",
"quote",
"syn 2.0.96",
"termcolor",
"thiserror 1.0.64",
"webpki-roots",
]
[[package]]
@@ -7056,10 +7058,28 @@ dependencies = [
]
[[package]]
name = "mysql_async"
version = "0.33.0"
name = "mysql-common-derive"
version = "0.32.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "6750b17ce50f8f112ef1a8394121090d47c596b56a6a17569ca680a9626e2ef2"
checksum = "deb6d9ff4094f6d58d3f892fc558e60048476213dd17dcf904b62202e9029da6"
dependencies = [
"darling 0.20.10",
"heck 0.5.0",
"num-bigint",
"proc-macro-crate 3.2.0",
"proc-macro-error2",
"proc-macro2",
"quote",
"syn 2.0.96",
"termcolor",
"thiserror 1.0.64",
]
[[package]]
name = "mysql_async"
version = "0.35.1"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "d14cf024116ba8fef4a7fec5abf0bd5de89b9fb29a7e55818a119ac5ec745077"
dependencies = [
"bytes",
"crossbeam",
@@ -7068,67 +7088,25 @@ dependencies = [
"futures-sink",
"futures-util",
"keyed_priority_queue",
"lazy_static",
"lru",
"mio 0.8.11",
"mysql_common 0.31.0",
"once_cell",
"mysql_common 0.34.1",
"pem",
"percent-encoding",
"pin-project",
"rand 0.8.5",
"rustls 0.21.12",
"rustls-pemfile 1.0.4",
"rustls",
"rustls-pemfile",
"serde",
"serde_json",
"socket2",
"thiserror 1.0.64",
"thiserror 2.0.12",
"tokio",
"tokio-rustls 0.24.1",
"tokio-rustls",
"tokio-util",
"twox-hash",
"twox-hash 2.1.0",
"url",
"webpki",
"webpki-roots 0.25.4",
]
[[package]]
name = "mysql_common"
version = "0.31.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "06f19e4cfa0ab5a76b627cec2d81331c49b034988eaf302c3bafeada684eadef"
dependencies = [
"base64 0.21.7",
"bigdecimal 0.4.5",
"bindgen",
"bitflags 2.9.0",
"bitvec",
"btoi",
"byteorder",
"bytes",
"cc",
"cmake",
"crc32fast",
"flate2",
"frunk",
"lazy_static",
"mysql-common-derive 0.30.2",
"num-bigint",
"num-traits",
"rand 0.8.5",
"regex",
"rust_decimal",
"saturating",
"serde",
"serde_json",
"sha1",
"sha2",
"smallvec",
"subprocess",
"thiserror 1.0.64",
"time",
"uuid",
"zstd 0.12.4",
"webpki-roots",
]
[[package]]
@@ -7171,6 +7149,39 @@ dependencies = [
"zstd 0.13.2",
]
[[package]]
name = "mysql_common"
version = "0.34.1"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "34a9141e735d5bb02414a7ac03add09522466d4db65bdd827069f76ae0850e58"
dependencies = [
"base64 0.22.1",
"bitflags 2.9.0",
"btoi",
"byteorder",
"bytes",
"cc",
"cmake",
"crc32fast",
"flate2",
"lazy_static",
"mysql-common-derive 0.32.0",
"num-bigint",
"num-traits",
"rand 0.8.5",
"regex",
"saturating",
"serde",
"serde_json",
"sha1",
"sha2",
"subprocess",
"thiserror 1.0.64",
"time",
"uuid",
"zstd 0.13.2",
]
[[package]]
name = "nalgebra"
version = "0.29.0"
@@ -7338,7 +7349,7 @@ dependencies = [
"kqueue",
"libc",
"log",
"mio 1.0.2",
"mio",
"notify-types",
"walkdir",
"windows-sys 0.59.0",
@@ -7638,8 +7649,8 @@ dependencies = [
[[package]]
name = "opensrv-mysql"
version = "0.7.0"
source = "git+https://github.com/datafuselabs/opensrv?rev=6bbc3b65e6b19212c4f7fc4f40c20daf6f452deb#6bbc3b65e6b19212c4f7fc4f40c20daf6f452deb"
version = "0.8.0"
source = "git+https://github.com/datafuselabs/opensrv?rev=a1fb4da215c8693c7e4f62be249a01b7fec52997#a1fb4da215c8693c7e4f62be249a01b7fec52997"
dependencies = [
"async-trait",
"byteorder",
@@ -7649,7 +7660,7 @@ dependencies = [
"nom",
"pin-project-lite",
"tokio",
"tokio-rustls 0.26.0",
"tokio-rustls",
]
[[package]]
@@ -8053,7 +8064,7 @@ dependencies = [
"snap",
"thrift",
"tokio",
"twox-hash",
"twox-hash 1.6.3",
"zstd 0.13.2",
"zstd-sys",
]
@@ -8083,6 +8094,7 @@ dependencies = [
"itertools 0.14.0",
"serde",
"serde_json",
"session",
"snafu 0.8.5",
"sql",
"sqlparser 0.52.0 (git+https://github.com/GreptimeTeam/sqlparser-rs.git?rev=71dd86058d2af97b9925093d40c4e03360403170)",
@@ -8261,7 +8273,7 @@ dependencies = [
"rust_decimal",
"thiserror 2.0.12",
"tokio",
"tokio-rustls 0.26.0",
"tokio-rustls",
"tokio-util",
]
@@ -8385,6 +8397,7 @@ dependencies = [
"serde",
"serde_json",
"session",
"simd-json",
"snafu 0.8.5",
"sql",
"table",
@@ -8692,6 +8705,7 @@ dependencies = [
"proc-macro-error-attr2",
"proc-macro2",
"quote",
"syn 2.0.96",
]
[[package]]
@@ -9094,6 +9108,7 @@ dependencies = [
"num-traits",
"object-store",
"once_cell",
"partition",
"paste",
"pretty_assertions",
"prometheus",
@@ -9158,7 +9173,7 @@ dependencies = [
"quinn-proto",
"quinn-udp",
"rustc-hash 2.0.0",
"rustls 0.23.20",
"rustls",
"socket2",
"thiserror 1.0.64",
"tokio",
@@ -9175,7 +9190,7 @@ dependencies = [
"rand 0.8.5",
"ring",
"rustc-hash 2.0.0",
"rustls 0.23.20",
"rustls",
"slab",
"thiserror 1.0.64",
"tinyvec",
@@ -9590,16 +9605,16 @@ dependencies = [
"percent-encoding",
"pin-project-lite",
"quinn",
"rustls 0.23.20",
"rustls",
"rustls-native-certs 0.8.0",
"rustls-pemfile 2.2.0",
"rustls-pemfile",
"rustls-pki-types",
"serde",
"serde_json",
"serde_urlencoded",
"sync_wrapper 1.0.1",
"tokio",
"tokio-rustls 0.26.0",
"tokio-rustls",
"tokio-util",
"tower-service",
"url",
@@ -9607,7 +9622,7 @@ dependencies = [
"wasm-bindgen-futures",
"wasm-streams",
"web-sys",
"webpki-roots 0.26.6",
"webpki-roots",
"windows-registry",
]
@@ -9758,11 +9773,11 @@ dependencies = [
"parking_lot 0.12.3",
"rand 0.8.5",
"rsasl",
"rustls 0.23.20",
"rustls",
"snap",
"thiserror 1.0.64",
"tokio",
"tokio-rustls 0.26.0",
"tokio-rustls",
"tracing",
"zstd 0.13.2",
]
@@ -9992,26 +10007,15 @@ dependencies = [
[[package]]
name = "rustls"
version = "0.21.12"
version = "0.23.25"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "3f56a14d1f48b391359b22f731fd4bd7e43c97f3c50eee276f3aa09c94784d3e"
dependencies = [
"log",
"ring",
"rustls-webpki 0.101.7",
"sct",
]
[[package]]
name = "rustls"
version = "0.23.20"
source = "git+https://github.com/GreptimeTeam/rustls?rev=34fd0c6#34fd0c6244af1501ca8f5b5c1c69afda67ce8fbb"
checksum = "822ee9188ac4ec04a2f0531e55d035fb2de73f18b41a63c70c2712503b6fb13c"
dependencies = [
"log",
"once_cell",
"ring",
"rustls-pki-types",
"rustls-webpki 0.102.8",
"rustls-webpki",
"subtle",
"zeroize",
]
@@ -10023,7 +10027,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "e5bfb394eeed242e909609f56089eecfe5fda225042e8b171791b9c95f5931e5"
dependencies = [
"openssl-probe",
"rustls-pemfile 2.2.0",
"rustls-pemfile",
"rustls-pki-types",
"schannel",
"security-framework",
@@ -10036,21 +10040,12 @@ source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "fcaf18a4f2be7326cd874a5fa579fae794320a0f388d365dca7e480e55f83f8a"
dependencies = [
"openssl-probe",
"rustls-pemfile 2.2.0",
"rustls-pemfile",
"rustls-pki-types",
"schannel",
"security-framework",
]
[[package]]
name = "rustls-pemfile"
version = "1.0.4"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "1c74cae0a4cf6ccbbf5f359f08efdf8ee7e1dc532573bf0db71968cb56b1448c"
dependencies = [
"base64 0.21.7",
]
[[package]]
name = "rustls-pemfile"
version = "2.2.0"
@@ -10062,25 +10057,15 @@ dependencies = [
[[package]]
name = "rustls-pki-types"
version = "1.10.1"
version = "1.11.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "d2bf47e6ff922db3825eb750c4e2ff784c6ff8fb9e13046ef6a1d1c5401b0b37"
checksum = "917ce264624a4b4db1c364dcc35bfca9ded014d0a958cd47ad3e960e988ea51c"
[[package]]
name = "rustls-webpki"
version = "0.101.7"
version = "0.103.1"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "8b6275d1ee7a1cd780b64aca7726599a1dbc893b1e64144529e55c3c2f745765"
dependencies = [
"ring",
"untrusted",
]
[[package]]
name = "rustls-webpki"
version = "0.102.8"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "64ca1bc8749bd4cf37b5ce386cc146580777b4e8572c7b97baf22c83f444bee9"
checksum = "fef8b8769aaccf73098557a87cd1816b4f9c7c16811c9c77142aa695c16f2c03"
dependencies = [
"ring",
"rustls-pki-types",
@@ -10272,16 +10257,6 @@ dependencies = [
"sha2",
]
[[package]]
name = "sct"
version = "0.7.1"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "da046153aa2352493d6cb7da4b6e5c0c057d8a1d0a9aa8560baffdd945acd414"
dependencies = [
"ring",
"untrusted",
]
[[package]]
name = "seahash"
version = "4.1.0"
@@ -10532,6 +10507,7 @@ dependencies = [
"common-query",
"common-recordbatch",
"common-runtime",
"common-session",
"common-telemetry",
"common-test-util",
"common-time",
@@ -10586,12 +10562,13 @@ dependencies = [
"regex",
"reqwest",
"rust-embed",
"rustls 0.23.20",
"rustls-pemfile 2.2.0",
"rustls",
"rustls-pemfile",
"rustls-pki-types",
"serde",
"serde_json",
"session",
"simd-json",
"snafu 0.8.5",
"snap",
"socket2",
@@ -10604,7 +10581,7 @@ dependencies = [
"tokio",
"tokio-postgres",
"tokio-postgres-rustls",
"tokio-rustls 0.26.0",
"tokio-rustls",
"tokio-stream",
"tokio-util",
"tonic 0.12.3",
@@ -10627,6 +10604,7 @@ dependencies = [
"common-error",
"common-macro",
"common-recordbatch",
"common-session",
"common-telemetry",
"common-time",
"derive_builder 0.20.1",
@@ -10730,6 +10708,21 @@ dependencies = [
"wide",
]
[[package]]
name = "simd-json"
version = "0.15.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "10b5602e4f1f7d358956f94cac1eff59220f34cf9e26d49f5fde5acef851cbed"
dependencies = [
"getrandom 0.3.2",
"halfbrown",
"ref-cast",
"serde",
"serde_json",
"simdutf8",
"value-trait",
]
[[package]]
name = "simdutf8"
version = "0.1.5"
@@ -11100,8 +11093,8 @@ dependencies = [
"memchr",
"once_cell",
"percent-encoding",
"rustls 0.23.20",
"rustls-pemfile 2.2.0",
"rustls",
"rustls-pemfile",
"serde",
"serde_json",
"sha2",
@@ -11111,7 +11104,7 @@ dependencies = [
"tokio-stream",
"tracing",
"url",
"webpki-roots 0.26.6",
"webpki-roots",
]
[[package]]
@@ -12221,7 +12214,7 @@ dependencies = [
"backtrace",
"bytes",
"libc",
"mio 1.0.2",
"mio",
"parking_lot 0.12.3",
"pin-project-lite",
"signal-hook-registry",
@@ -12310,30 +12303,20 @@ source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "04fb792ccd6bbcd4bba408eb8a292f70fc4a3589e5d793626f45190e6454b6ab"
dependencies = [
"ring",
"rustls 0.23.20",
"rustls",
"tokio",
"tokio-postgres",
"tokio-rustls 0.26.0",
"tokio-rustls",
"x509-certificate",
]
[[package]]
name = "tokio-rustls"
version = "0.24.1"
version = "0.26.2"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "c28327cf380ac148141087fbfb9de9d7bd4e84ab5d2c28fbc911d753de8a7081"
checksum = "8e727b36a1a0e8b74c376ac2211e40c2c8af09fb4013c60d910495810f008e9b"
dependencies = [
"rustls 0.21.12",
"tokio",
]
[[package]]
name = "tokio-rustls"
version = "0.26.0"
source = "git+https://github.com/GreptimeTeam/tokio-rustls?rev=4604ca6#4604ca6badfd1d10424718e5570cc481ab787fc8"
dependencies = [
"rustls 0.23.20",
"rustls-pki-types",
"rustls",
"tokio",
]
@@ -12481,10 +12464,10 @@ dependencies = [
"percent-encoding",
"pin-project",
"prost 0.13.3",
"rustls-pemfile 2.2.0",
"rustls-pemfile",
"socket2",
"tokio",
"tokio-rustls 0.26.0",
"tokio-rustls",
"tokio-stream",
"tower 0.4.13",
"tower-layer",
@@ -12881,10 +12864,15 @@ source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "97fee6b57c6a41524a810daee9286c02d7752c4253064d0b05472833a438f675"
dependencies = [
"cfg-if",
"rand 0.8.5",
"static_assertions",
]
[[package]]
name = "twox-hash"
version = "2.1.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "e7b17f197b3050ba473acf9181f7b1d3b66d1cf7356c6cc57886662276e65908"
[[package]]
name = "typeid"
version = "1.0.2"
@@ -13194,6 +13182,18 @@ version = "0.1.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "830b7e5d4d90034032940e4ace0d9a9a057e7a45cd94e6c007832e39edb82f6d"
[[package]]
name = "value-trait"
version = "0.11.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "0508fce11ad19e0aab49ce20b6bec7f8f82902ded31df1c9fc61b90f0eb396b8"
dependencies = [
"float-cmp",
"halfbrown",
"itoa",
"ryu",
]
[[package]]
name = "variadics"
version = "0.0.4"
@@ -13284,24 +13284,24 @@ checksum = "b8dad83b4f25e74f184f64c43b150b91efe7647395b42289f38e50566d82855b"
[[package]]
name = "wasm-bindgen"
version = "0.2.93"
version = "0.2.100"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "a82edfc16a6c469f5f44dc7b571814045d60404b55a0ee849f9bcfa2e63dd9b5"
checksum = "1edc8929d7499fc4e8f0be2262a241556cfc54a0bea223790e71446f2aab1ef5"
dependencies = [
"cfg-if",
"once_cell",
"rustversion",
"wasm-bindgen-macro",
]
[[package]]
name = "wasm-bindgen-backend"
version = "0.2.93"
version = "0.2.100"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "9de396da306523044d3302746f1208fa71d7532227f15e347e2d93e4145dd77b"
checksum = "2f0a0651a5c2bc21487bde11ee802ccaf4c51935d0d3d42a6101f98161700bc6"
dependencies = [
"bumpalo",
"log",
"once_cell",
"proc-macro2",
"quote",
"syn 2.0.96",
@@ -13322,9 +13322,9 @@ dependencies = [
[[package]]
name = "wasm-bindgen-macro"
version = "0.2.93"
version = "0.2.100"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "585c4c91a46b072c92e908d99cb1dcdf95c5218eeb6f3bf1efa991ee7a68cccf"
checksum = "7fe63fc6d09ed3792bd0897b314f53de8e16568c2b3f7982f468c0bf9bd0b407"
dependencies = [
"quote",
"wasm-bindgen-macro-support",
@@ -13332,9 +13332,9 @@ dependencies = [
[[package]]
name = "wasm-bindgen-macro-support"
version = "0.2.93"
version = "0.2.100"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "afc340c74d9005395cf9dd098506f7f44e38f2b4a21c6aaacf9a105ea5e1e836"
checksum = "8ae87ea40c9f689fc23f209965b6fb8a99ad69aeeb0231408be24920604395de"
dependencies = [
"proc-macro2",
"quote",
@@ -13345,9 +13345,12 @@ dependencies = [
[[package]]
name = "wasm-bindgen-shared"
version = "0.2.93"
version = "0.2.100"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "c62a0a307cb4a311d3a07867860911ca130c3494e8c2719593806c08bc5d0484"
checksum = "1a05d73b933a847d6cccdda8f838a22ff101ad9bf93e33684f39c1f5f0eece3d"
dependencies = [
"unicode-ident",
]
[[package]]
name = "wasm-streams"
@@ -13419,12 +13422,6 @@ dependencies = [
"untrusted",
]
[[package]]
name = "webpki-roots"
version = "0.25.4"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "5f20c57d8d7db6d3b86154206ae5d8fba62dd39573114de97c2cb0578251f8e1"
[[package]]
name = "webpki-roots"
version = "0.26.6"
@@ -14100,15 +14097,6 @@ dependencies = [
"zstd-safe 5.0.2+zstd.1.5.2",
]
[[package]]
name = "zstd"
version = "0.12.4"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "1a27595e173641171fc74a1232b7b1c7a7cb6e18222c11e9dfb9888fa424c53c"
dependencies = [
"zstd-safe 6.0.6",
]
[[package]]
name = "zstd"
version = "0.13.2"
@@ -14128,16 +14116,6 @@ dependencies = [
"zstd-sys",
]
[[package]]
name = "zstd-safe"
version = "6.0.6"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "ee98ffd0b48ee95e6c5168188e44a54550b1564d9d530ee21d5f0eaed1069581"
dependencies = [
"libc",
"zstd-sys",
]
[[package]]
name = "zstd-safe"
version = "7.2.1"

View File

@@ -29,6 +29,7 @@ members = [
"src/common/query",
"src/common/recordbatch",
"src/common/runtime",
"src/common/session",
"src/common/substrait",
"src/common/telemetry",
"src/common/test-util",
@@ -129,7 +130,7 @@ etcd-client = "0.14"
fst = "0.4.7"
futures = "0.3"
futures-util = "0.3"
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "97e298d119fdb9499bc6ba9e03f375cfa7cdf130" }
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "dd4a1996982534636734674db66e44464b0c0d83" }
hex = "0.4"
http = "1"
humantime = "2.1"
@@ -180,11 +181,13 @@ rstest = "0.25"
rstest_reuse = "0.7"
rust_decimal = "1.33"
rustc-hash = "2.0"
rustls = { version = "0.23.20", default-features = false } # override by patch, see [patch.crates-io]
# It is worth noting that we should try to avoid using aws-lc-rs until it can be compiled on various platforms.
rustls = { version = "0.23.25", default-features = false }
serde = { version = "1.0", features = ["derive"] }
serde_json = { version = "1.0", features = ["float_roundtrip"] }
serde_with = "3"
shadow-rs = "1.1"
simd-json = "0.15"
similar-asserts = "1.6.0"
smallvec = { version = "1", features = ["serde"] }
snafu = "0.8"
@@ -204,7 +207,7 @@ strum = { version = "0.27", features = ["derive"] }
tempfile = "3"
tokio = { version = "1.40", features = ["full"] }
tokio-postgres = "0.7"
tokio-rustls = { version = "0.26.0", default-features = false } # override by patch, see [patch.crates-io]
tokio-rustls = { version = "0.26.2", default-features = false }
tokio-stream = "0.1"
tokio-util = { version = "0.7", features = ["io-util", "compat"] }
toml = "0.8.8"
@@ -247,6 +250,7 @@ common-procedure-test = { path = "src/common/procedure-test" }
common-query = { path = "src/common/query" }
common-recordbatch = { path = "src/common/recordbatch" }
common-runtime = { path = "src/common/runtime" }
common-session = { path = "src/common/session" }
common-telemetry = { path = "src/common/telemetry" }
common-test-util = { path = "src/common/test-util" }
common-time = { path = "src/common/time" }
@@ -279,15 +283,6 @@ store-api = { path = "src/store-api" }
substrait = { path = "src/common/substrait" }
table = { path = "src/table" }
[patch.crates-io]
# change all rustls dependencies to use our fork to default to `ring` to make it "just work"
hyper-rustls = { git = "https://github.com/GreptimeTeam/hyper-rustls", rev = "a951e03" } # version = "0.27.5" with ring patch
rustls = { git = "https://github.com/GreptimeTeam/rustls", rev = "34fd0c6" } # version = "0.23.20" with ring patch
tokio-rustls = { git = "https://github.com/GreptimeTeam/tokio-rustls", rev = "4604ca6" } # version = "0.26.0" with ring patch
# This is commented, since we are not using aws-lc-sys, if we need to use it, we need to uncomment this line or use a release after this commit, or it wouldn't compile with gcc < 8.1
# see https://github.com/aws/aws-lc-rs/pull/526
# aws-lc-sys = { git ="https://github.com/aws/aws-lc-rs", rev = "556558441e3494af4b156ae95ebc07ebc2fd38aa" }
[workspace.dependencies.meter-macros]
git = "https://github.com/GreptimeTeam/greptime-meter.git"
rev = "5618e779cf2bb4755b499c630fba4c35e91898cb"

View File

@@ -66,7 +66,7 @@
## News
**[GreptimeDB archives 1 billion cold run #1 in JSONBench!](https://greptime.com/blogs/2025-03-18-jsonbench-greptimedb-performance)**
**[GreptimeDB tops JSONBench's billion-record cold run test!](https://greptime.com/blogs/2025-03-18-jsonbench-greptimedb-performance)**
## Why GreptimeDB

View File

@@ -12,7 +12,6 @@
| Key | Type | Default | Descriptions |
| --- | -----| ------- | ----------- |
| `mode` | String | `standalone` | The running mode of the datanode. It can be `standalone` or `distributed`. |
| `default_timezone` | String | Unset | The default timezone of the server. |
| `init_regions_in_background` | Bool | `false` | Initialize all regions in the background during the startup.<br/>By default, it provides services after all regions have been initialized. |
| `init_regions_parallelism` | Integer | `16` | Parallelism of initializing regions. |
@@ -86,10 +85,6 @@
| `wal.create_topic_timeout` | String | `30s` | Above which a topic creation operation will be cancelled.<br/>**It's only used when the provider is `kafka`**. |
| `wal.max_batch_bytes` | String | `1MB` | The max size of a single producer batch.<br/>Warning: Kafka has a default limit of 1MB per message in a topic.<br/>**It's only used when the provider is `kafka`**. |
| `wal.consumer_wait_timeout` | String | `100ms` | The consumer wait timeout.<br/>**It's only used when the provider is `kafka`**. |
| `wal.backoff_init` | String | `500ms` | The initial backoff delay.<br/>**It's only used when the provider is `kafka`**. |
| `wal.backoff_max` | String | `10s` | The maximum backoff delay.<br/>**It's only used when the provider is `kafka`**. |
| `wal.backoff_base` | Integer | `2` | The exponential backoff rate, i.e. next backoff = base * current backoff.<br/>**It's only used when the provider is `kafka`**. |
| `wal.backoff_deadline` | String | `5mins` | The deadline of retries.<br/>**It's only used when the provider is `kafka`**. |
| `wal.overwrite_entry_start_id` | Bool | `false` | Ignore missing entries during read WAL.<br/>**It's only used when the provider is `kafka`**.<br/><br/>This option ensures that when Kafka messages are deleted, the system<br/>can still successfully replay memtable data without throwing an<br/>out-of-range error.<br/>However, enabling this option might lead to unexpected data loss,<br/>as the system will skip over missing entries instead of treating<br/>them as critical errors. |
| `metadata_store` | -- | -- | Metadata storage options. |
| `metadata_store.file_size` | String | `64MB` | The size of the metadata store log file. |
@@ -98,6 +93,7 @@
| `procedure` | -- | -- | Procedure storage options. |
| `procedure.max_retry_times` | Integer | `3` | Procedure max retry time. |
| `procedure.retry_delay` | String | `500ms` | Initial retry delay of procedures, increases exponentially |
| `procedure.max_running_procedures` | Integer | `128` | Max running procedures.<br/>The maximum number of procedures that can be running at the same time.<br/>If the number of running procedures exceeds this limit, the procedure will be rejected. |
| `flow` | -- | -- | flow engine options. |
| `flow.num_workers` | Integer | `0` | The number of flow worker in flownode.<br/>Not setting(or set to 0) this value will use the number of CPU cores divided by 2. |
| `storage` | -- | -- | The data storage options. |
@@ -328,6 +324,7 @@
| `procedure.max_retry_times` | Integer | `12` | Procedure max retry time. |
| `procedure.retry_delay` | String | `500ms` | Initial retry delay of procedures, increases exponentially |
| `procedure.max_metadata_value_size` | String | `1500KiB` | Auto split large value<br/>GreptimeDB procedure uses etcd as the default metadata storage backend.<br/>The etcd the maximum size of any request is 1.5 MiB<br/>1500KiB = 1536KiB (1.5MiB) - 36KiB (reserved size of key)<br/>Comments out the `max_metadata_value_size`, for don't split large value (no limit). |
| `procedure.max_running_procedures` | Integer | `128` | Max running procedures.<br/>The maximum number of procedures that can be running at the same time.<br/>If the number of running procedures exceeds this limit, the procedure will be rejected. |
| `failure_detector` | -- | -- | -- |
| `failure_detector.threshold` | Float | `8.0` | The threshold value used by the failure detector to determine failure conditions. |
| `failure_detector.min_std_deviation` | String | `100ms` | The minimum standard deviation of the heartbeat intervals, used to calculate acceptable variations. |
@@ -347,10 +344,6 @@
| `wal.topic_name_prefix` | String | `greptimedb_wal_topic` | A Kafka topic is constructed by concatenating `topic_name_prefix` and `topic_id`.<br/>Only accepts strings that match the following regular expression pattern:<br/>[a-zA-Z_:-][a-zA-Z0-9_:\-\.@#]*<br/>i.g., greptimedb_wal_topic_0, greptimedb_wal_topic_1. |
| `wal.replication_factor` | Integer | `1` | Expected number of replicas of each partition. |
| `wal.create_topic_timeout` | String | `30s` | Above which a topic creation operation will be cancelled. |
| `wal.backoff_init` | String | `500ms` | The initial backoff for kafka clients. |
| `wal.backoff_max` | String | `10s` | The maximum backoff for kafka clients. |
| `wal.backoff_base` | Integer | `2` | Exponential backoff rate, i.e. next backoff = base * current backoff. |
| `wal.backoff_deadline` | String | `5mins` | Stop reconnecting if the total wait time reaches the deadline. If this config is missing, the reconnecting won't terminate. |
| `logging` | -- | -- | The logging options. |
| `logging.dir` | String | `./greptimedb_data/logs` | The directory to store the log files. If set to empty, logs will not be written to files. |
| `logging.level` | String | Unset | The log level. Can be `info`/`debug`/`warn`/`error`. |
@@ -381,7 +374,6 @@
| Key | Type | Default | Descriptions |
| --- | -----| ------- | ----------- |
| `mode` | String | `standalone` | The running mode of the datanode. It can be `standalone` or `distributed`. |
| `node_id` | Integer | Unset | The datanode identifier and should be unique in the cluster. |
| `require_lease_before_startup` | Bool | `false` | Start services after regions have obtained leases.<br/>It will block the datanode start if it can't receive leases in the heartbeat from metasrv. |
| `init_regions_in_background` | Bool | `false` | Initialize all regions in the background during the startup.<br/>By default, it provides services after all regions have been initialized. |
@@ -434,10 +426,6 @@
| `wal.broker_endpoints` | Array | -- | The Kafka broker endpoints.<br/>**It's only used when the provider is `kafka`**. |
| `wal.max_batch_bytes` | String | `1MB` | The max size of a single producer batch.<br/>Warning: Kafka has a default limit of 1MB per message in a topic.<br/>**It's only used when the provider is `kafka`**. |
| `wal.consumer_wait_timeout` | String | `100ms` | The consumer wait timeout.<br/>**It's only used when the provider is `kafka`**. |
| `wal.backoff_init` | String | `500ms` | The initial backoff delay.<br/>**It's only used when the provider is `kafka`**. |
| `wal.backoff_max` | String | `10s` | The maximum backoff delay.<br/>**It's only used when the provider is `kafka`**. |
| `wal.backoff_base` | Integer | `2` | The exponential backoff rate, i.e. next backoff = base * current backoff.<br/>**It's only used when the provider is `kafka`**. |
| `wal.backoff_deadline` | String | `5mins` | The deadline of retries.<br/>**It's only used when the provider is `kafka`**. |
| `wal.create_index` | Bool | `true` | Whether to enable WAL index creation.<br/>**It's only used when the provider is `kafka`**. |
| `wal.dump_index_interval` | String | `60s` | The interval for dumping WAL indexes.<br/>**It's only used when the provider is `kafka`**. |
| `wal.overwrite_entry_start_id` | Bool | `false` | Ignore missing entries during read WAL.<br/>**It's only used when the provider is `kafka`**.<br/><br/>This option ensures that when Kafka messages are deleted, the system<br/>can still successfully replay memtable data without throwing an<br/>out-of-range error.<br/>However, enabling this option might lead to unexpected data loss,<br/>as the system will skip over missing entries instead of treating<br/>them as critical errors. |
@@ -551,7 +539,6 @@
| Key | Type | Default | Descriptions |
| --- | -----| ------- | ----------- |
| `mode` | String | `distributed` | The running mode of the flownode. It can be `standalone` or `distributed`. |
| `node_id` | Integer | Unset | The flownode identifier and should be unique in the cluster. |
| `flow` | -- | -- | flow engine options. |
| `flow.num_workers` | Integer | `0` | The number of flow worker in flownode.<br/>Not setting(or set to 0) this value will use the number of CPU cores divided by 2. |

View File

@@ -1,6 +1,3 @@
## The running mode of the datanode. It can be `standalone` or `distributed`.
mode = "standalone"
## The datanode identifier and should be unique in the cluster.
## @toml2docs:none-default
node_id = 42
@@ -169,22 +166,6 @@ max_batch_bytes = "1MB"
## **It's only used when the provider is `kafka`**.
consumer_wait_timeout = "100ms"
## The initial backoff delay.
## **It's only used when the provider is `kafka`**.
backoff_init = "500ms"
## The maximum backoff delay.
## **It's only used when the provider is `kafka`**.
backoff_max = "10s"
## The exponential backoff rate, i.e. next backoff = base * current backoff.
## **It's only used when the provider is `kafka`**.
backoff_base = 2
## The deadline of retries.
## **It's only used when the provider is `kafka`**.
backoff_deadline = "5mins"
## Whether to enable WAL index creation.
## **It's only used when the provider is `kafka`**.
create_index = true

View File

@@ -1,6 +1,3 @@
## The running mode of the flownode. It can be `standalone` or `distributed`.
mode = "distributed"
## The flownode identifier and should be unique in the cluster.
## @toml2docs:none-default
node_id = 14

View File

@@ -79,6 +79,11 @@ retry_delay = "500ms"
## Comments out the `max_metadata_value_size`, for don't split large value (no limit).
max_metadata_value_size = "1500KiB"
## Max running procedures.
## The maximum number of procedures that can be running at the same time.
## If the number of running procedures exceeds this limit, the procedure will be rejected.
max_running_procedures = 128
# Failure detectors options.
[failure_detector]
@@ -144,17 +149,6 @@ replication_factor = 1
## Above which a topic creation operation will be cancelled.
create_topic_timeout = "30s"
## The initial backoff for kafka clients.
backoff_init = "500ms"
## The maximum backoff for kafka clients.
backoff_max = "10s"
## Exponential backoff rate, i.e. next backoff = base * current backoff.
backoff_base = 2
## Stop reconnecting if the total wait time reaches the deadline. If this config is missing, the reconnecting won't terminate.
backoff_deadline = "5mins"
# The Kafka SASL configuration.
# **It's only used when the provider is `kafka`**.

View File

@@ -1,6 +1,3 @@
## The running mode of the datanode. It can be `standalone` or `distributed`.
mode = "standalone"
## The default timezone of the server.
## @toml2docs:none-default
default_timezone = "UTC"
@@ -242,22 +239,6 @@ max_batch_bytes = "1MB"
## **It's only used when the provider is `kafka`**.
consumer_wait_timeout = "100ms"
## The initial backoff delay.
## **It's only used when the provider is `kafka`**.
backoff_init = "500ms"
## The maximum backoff delay.
## **It's only used when the provider is `kafka`**.
backoff_max = "10s"
## The exponential backoff rate, i.e. next backoff = base * current backoff.
## **It's only used when the provider is `kafka`**.
backoff_base = 2
## The deadline of retries.
## **It's only used when the provider is `kafka`**.
backoff_deadline = "5mins"
## Ignore missing entries during read WAL.
## **It's only used when the provider is `kafka`**.
##
@@ -302,6 +283,10 @@ purge_interval = "1m"
max_retry_times = 3
## Initial retry delay of procedures, increases exponentially
retry_delay = "500ms"
## Max running procedures.
## The maximum number of procedures that can be running at the same time.
## If the number of running procedures exceeds this limit, the procedure will be rejected.
max_running_procedures = 128
## flow engine options.
[flow]

View File

@@ -15,10 +15,13 @@
use std::collections::HashMap;
use datatypes::schema::{
ColumnDefaultConstraint, ColumnSchema, FulltextAnalyzer, FulltextOptions, SkippingIndexOptions,
SkippingIndexType, COMMENT_KEY, FULLTEXT_KEY, INVERTED_INDEX_KEY, SKIPPING_INDEX_KEY,
ColumnDefaultConstraint, ColumnSchema, FulltextAnalyzer, FulltextBackend, FulltextOptions,
SkippingIndexOptions, SkippingIndexType, COMMENT_KEY, FULLTEXT_KEY, INVERTED_INDEX_KEY,
SKIPPING_INDEX_KEY,
};
use greptime_proto::v1::{
Analyzer, FulltextBackend as PbFulltextBackend, SkippingIndexType as PbSkippingIndexType,
};
use greptime_proto::v1::{Analyzer, SkippingIndexType as PbSkippingIndexType};
use snafu::ResultExt;
use crate::error::{self, Result};
@@ -142,13 +145,21 @@ pub fn options_from_inverted() -> ColumnOptions {
}
/// Tries to construct a `FulltextAnalyzer` from the given analyzer.
pub fn as_fulltext_option(analyzer: Analyzer) -> FulltextAnalyzer {
pub fn as_fulltext_option_analyzer(analyzer: Analyzer) -> FulltextAnalyzer {
match analyzer {
Analyzer::English => FulltextAnalyzer::English,
Analyzer::Chinese => FulltextAnalyzer::Chinese,
}
}
/// Tries to construct a `FulltextBackend` from the given backend.
pub fn as_fulltext_option_backend(backend: PbFulltextBackend) -> FulltextBackend {
match backend {
PbFulltextBackend::Bloom => FulltextBackend::Bloom,
PbFulltextBackend::Tantivy => FulltextBackend::Tantivy,
}
}
/// Tries to construct a `SkippingIndexType` from the given skipping index type.
pub fn as_skipping_index_type(skipping_index_type: PbSkippingIndexType) -> SkippingIndexType {
match skipping_index_type {
@@ -160,7 +171,7 @@ pub fn as_skipping_index_type(skipping_index_type: PbSkippingIndexType) -> Skipp
mod tests {
use datatypes::data_type::ConcreteDataType;
use datatypes::schema::FulltextAnalyzer;
use datatypes::schema::{FulltextAnalyzer, FulltextBackend};
use super::*;
use crate::v1::ColumnDataType;
@@ -219,13 +230,14 @@ mod tests {
enable: true,
analyzer: FulltextAnalyzer::English,
case_sensitive: false,
backend: FulltextBackend::Bloom,
})
.unwrap();
schema.set_inverted_index(true);
let options = options_from_column_schema(&schema).unwrap();
assert_eq!(
options.options.get(FULLTEXT_GRPC_KEY).unwrap(),
"{\"enable\":true,\"analyzer\":\"English\",\"case-sensitive\":false}"
"{\"enable\":true,\"analyzer\":\"English\",\"case-sensitive\":false,\"backend\":\"bloom\"}"
);
assert_eq!(
options.options.get(INVERTED_INDEX_GRPC_KEY).unwrap(),
@@ -239,11 +251,12 @@ mod tests {
enable: true,
analyzer: FulltextAnalyzer::English,
case_sensitive: false,
backend: FulltextBackend::Bloom,
};
let options = options_from_fulltext(&fulltext).unwrap().unwrap();
assert_eq!(
options.options.get(FULLTEXT_GRPC_KEY).unwrap(),
"{\"enable\":true,\"analyzer\":\"English\",\"case-sensitive\":false}"
"{\"enable\":true,\"analyzer\":\"English\",\"case-sensitive\":false,\"backend\":\"bloom\"}"
);
}

View File

@@ -30,7 +30,7 @@ use datanode::datanode::{Datanode, DatanodeBuilder};
use datanode::service::DatanodeServiceBuilder;
use meta_client::{MetaClientOptions, MetaClientType};
use servers::Mode;
use snafu::{OptionExt, ResultExt};
use snafu::{ensure, OptionExt, ResultExt};
use tracing_appender::non_blocking::WorkerGuard;
use crate::error::{
@@ -223,15 +223,14 @@ impl StartCommand {
.get_or_insert_with(MetaClientOptions::default)
.metasrv_addrs
.clone_from(metasrv_addrs);
opts.mode = Mode::Distributed;
}
if let (Mode::Distributed, None) = (&opts.mode, &opts.node_id) {
return MissingConfigSnafu {
msg: "Missing node id option",
ensure!(
opts.node_id.is_some(),
MissingConfigSnafu {
msg: "Missing node id option"
}
.fail();
}
);
if let Some(data_home) = &self.data_home {
opts.storage.data_home.clone_from(data_home);
@@ -314,7 +313,7 @@ impl StartCommand {
.build(),
);
let mut datanode = DatanodeBuilder::new(opts.clone(), plugins)
let mut datanode = DatanodeBuilder::new(opts.clone(), plugins, Mode::Distributed)
.with_meta_client(meta_client)
.with_kv_backend(meta_backend)
.with_cache_registry(layered_cache_registry)
@@ -336,6 +335,7 @@ impl StartCommand {
#[cfg(test)]
mod tests {
use std::assert_matches::assert_matches;
use std::io::Write;
use std::time::Duration;
@@ -343,7 +343,6 @@ mod tests {
use common_test_util::temp_dir::create_named_temp_file;
use datanode::config::{FileConfig, GcsConfig, ObjectStoreConfig, S3Config};
use servers::heartbeat_options::HeartbeatOptions;
use servers::Mode;
use super::*;
use crate::options::GlobalOptions;
@@ -494,22 +493,6 @@ mod tests {
#[test]
fn test_try_from_cmd() {
let opt = StartCommand::default()
.load_options(&GlobalOptions::default())
.unwrap()
.component;
assert_eq!(Mode::Standalone, opt.mode);
let opt = (StartCommand {
node_id: Some(42),
metasrv_addrs: Some(vec!["127.0.0.1:3002".to_string()]),
..Default::default()
})
.load_options(&GlobalOptions::default())
.unwrap()
.component;
assert_eq!(Mode::Distributed, opt.mode);
assert!((StartCommand {
metasrv_addrs: Some(vec!["127.0.0.1:3002".to_string()]),
..Default::default()
@@ -528,7 +511,19 @@ mod tests {
#[test]
fn test_load_log_options_from_cli() {
let cmd = StartCommand::default();
let mut cmd = StartCommand::default();
let result = cmd.load_options(&GlobalOptions {
log_dir: Some("./greptimedb_data/test/logs".to_string()),
log_level: Some("debug".to_string()),
#[cfg(feature = "tokio-console")]
tokio_console_addr: None,
});
// Missing node_id.
assert_matches!(result, Err(crate::error::Error::MissingConfig { .. }));
cmd.node_id = Some(42);
let options = cmd
.load_options(&GlobalOptions {

View File

@@ -34,8 +34,7 @@ use common_telemetry::logging::TracingOptions;
use common_version::{short_version, version};
use flow::{FlownodeBuilder, FlownodeInstance, FrontendInvoker};
use meta_client::{MetaClientOptions, MetaClientType};
use servers::Mode;
use snafu::{OptionExt, ResultExt};
use snafu::{ensure, OptionExt, ResultExt};
use tracing_appender::non_blocking::WorkerGuard;
use crate::error::{
@@ -203,7 +202,6 @@ impl StartCommand {
.get_or_insert_with(MetaClientOptions::default)
.metasrv_addrs
.clone_from(metasrv_addrs);
opts.mode = Mode::Distributed;
}
if let Some(http_addr) = &self.http_addr {
@@ -214,12 +212,12 @@ impl StartCommand {
opts.http.timeout = Duration::from_secs(http_timeout);
}
if let (Mode::Distributed, None) = (&opts.mode, &opts.node_id) {
return MissingConfigSnafu {
msg: "Missing node id option",
ensure!(
opts.node_id.is_some(),
MissingConfigSnafu {
msg: "Missing node id option"
}
.fail();
}
);
Ok(())
}

View File

@@ -42,6 +42,7 @@ use common_meta::kv_backend::KvBackendRef;
use common_meta::node_manager::NodeManagerRef;
use common_meta::peer::Peer;
use common_meta::region_keeper::MemoryRegionKeeper;
use common_meta::region_registry::LeaderRegionRegistry;
use common_meta::sequence::SequenceBuilder;
use common_meta::wal_options_allocator::{build_wal_options_allocator, WalOptionsAllocatorRef};
use common_procedure::{ProcedureInfo, ProcedureManagerRef};
@@ -126,7 +127,6 @@ impl SubCommand {
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)]
#[serde(default)]
pub struct StandaloneOptions {
pub mode: Mode,
pub enable_telemetry: bool,
pub default_timezone: Option<String>,
pub http: HttpOptions,
@@ -156,7 +156,6 @@ pub struct StandaloneOptions {
impl Default for StandaloneOptions {
fn default() -> Self {
Self {
mode: Mode::Standalone,
enable_telemetry: true,
default_timezone: None,
http: HttpOptions::default(),
@@ -237,7 +236,6 @@ impl StandaloneOptions {
grpc: cloned_opts.grpc,
init_regions_in_background: cloned_opts.init_regions_in_background,
init_regions_parallelism: cloned_opts.init_regions_parallelism,
mode: Mode::Standalone,
..Default::default()
}
}
@@ -381,9 +379,6 @@ impl StartCommand {
global_options: &GlobalOptions,
opts: &mut StandaloneOptions,
) -> Result<()> {
// Should always be standalone mode.
opts.mode = Mode::Standalone;
if let Some(dir) = &global_options.log_dir {
opts.logging.dir.clone_from(dir);
}
@@ -508,7 +503,7 @@ impl StartCommand {
.build(),
);
let datanode = DatanodeBuilder::new(dn_opts, plugins.clone())
let datanode = DatanodeBuilder::new(dn_opts, plugins.clone(), Mode::Standalone)
.with_kv_backend(kv_backend.clone())
.with_cache_registry(layered_cache_registry.clone())
.build()
@@ -670,6 +665,7 @@ impl StartCommand {
node_manager,
cache_invalidator,
memory_region_keeper: Arc::new(MemoryRegionKeeper::default()),
leader_region_registry: Arc::new(LeaderRegionRegistry::default()),
table_metadata_manager,
table_metadata_allocator,
flow_metadata_manager,
@@ -787,6 +783,7 @@ impl InformationExtension for StandaloneInformationExtension {
manifest_size: region_stat.manifest_size,
sst_size: region_stat.sst_size,
index_size: region_stat.index_size,
region_manifest: region_stat.manifest.into(),
}
})
.collect::<Vec<_>>();
@@ -1063,7 +1060,6 @@ mod tests {
let options =
StandaloneOptions::load_layered_options(None, "GREPTIMEDB_STANDALONE").unwrap();
let default_options = StandaloneOptions::default();
assert_eq!(options.mode, default_options.mode);
assert_eq!(options.enable_telemetry, default_options.enable_telemetry);
assert_eq!(options.http, default_options.http);
assert_eq!(options.grpc, default_options.grpc);

View File

@@ -168,8 +168,8 @@ fn test_load_metasrv_example_config() {
tracing_sample_ratio: Some(Default::default()),
slow_query: SlowQueryOptions {
enable: false,
threshold: Some(Duration::from_secs(10)),
sample_ratio: Some(1.0),
threshold: None,
sample_ratio: None,
},
..Default::default()
},

View File

@@ -137,4 +137,12 @@ pub const SPAN_ID_COLUMN: &str = "span_id";
pub const SPAN_NAME_COLUMN: &str = "span_name";
pub const SERVICE_NAME_COLUMN: &str = "service_name";
pub const PARENT_SPAN_ID_COLUMN: &str = "parent_span_id";
pub const TRACE_TABLE_NAME: &str = "opentelemetry_traces";
pub const TRACE_TABLE_NAME_SESSION_KEY: &str = "trace_table_name";
// ---- End of special table and fields ----
/// Generate the trace services table name from the trace table name by adding `_services` suffix.
pub fn trace_services_table_name(trace_table_name: &str) -> String {
format!("{}_services", trace_table_name)
}
// ---- End of special table and fields ----

View File

@@ -39,6 +39,7 @@ geohash = { version = "0.13", optional = true }
h3o = { version = "0.6", optional = true }
hyperloglogplus = "0.4"
jsonb.workspace = true
memchr = "2.7"
nalgebra.workspace = true
num = "0.4"
num-traits = "0.2"

View File

@@ -27,6 +27,7 @@ use crate::scalars::hll_count::HllCalcFunction;
use crate::scalars::ip::IpFunctions;
use crate::scalars::json::JsonFunction;
use crate::scalars::matches::MatchesFunction;
use crate::scalars::matches_term::MatchesTermFunction;
use crate::scalars::math::MathFunction;
use crate::scalars::timestamp::TimestampFunction;
use crate::scalars::uddsketch_calc::UddSketchCalcFunction;
@@ -116,6 +117,7 @@ pub static FUNCTION_REGISTRY: Lazy<Arc<FunctionRegistry>> = Lazy::new(|| {
// Full text search function
MatchesFunction::register(&function_registry);
MatchesTermFunction::register(&function_registry);
// System and administration functions
SystemFunction::register(&function_registry);

View File

@@ -19,6 +19,7 @@ pub mod expression;
pub mod geo;
pub mod json;
pub mod matches;
pub mod matches_term;
pub mod math;
pub mod vector;

View File

@@ -0,0 +1,375 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use std::sync::Arc;
use std::{fmt, iter};
use common_query::error::{InvalidFuncArgsSnafu, Result};
use common_query::prelude::Volatility;
use datatypes::prelude::ConcreteDataType;
use datatypes::scalars::ScalarVectorBuilder;
use datatypes::vectors::{BooleanVector, BooleanVectorBuilder, MutableVector, VectorRef};
use memchr::memmem;
use snafu::ensure;
use crate::function::{Function, FunctionContext};
use crate::function_registry::FunctionRegistry;
/// Exact term/phrase matching function for text columns.
///
/// This function checks if a text column contains exact term/phrase matches
/// with non-alphanumeric boundaries. Designed for:
/// - Whole-word matching (e.g. "cat" in "cat!" but not in "category")
/// - Phrase matching (e.g. "hello world" in "note:hello world!")
///
/// # Signature
/// `matches_term(text: String, term: String) -> Boolean`
///
/// # Arguments
/// * `text` - String column to search
/// * `term` - Search term/phrase
///
/// # Returns
/// BooleanVector where each element indicates if the corresponding text
/// contains an exact match of the term, following these rules:
/// 1. Exact substring match found (case-sensitive)
/// 2. Match boundaries are either:
/// - Start/end of text
/// - Any non-alphanumeric character (including spaces, hyphens, punctuation, etc.)
///
/// # Examples
/// ```
/// -- SQL examples --
/// -- Match phrase with space --
/// SELECT matches_term(column, 'hello world') FROM table;
/// -- Text: "warning:hello world!" => true
/// -- Text: "hello-world" => false (hyphen instead of space)
/// -- Text: "hello world2023" => false (ending with numbers)
///
/// -- Match multiple words with boundaries --
/// SELECT matches_term(column, 'critical error') FROM logs;
/// -- Match in: "ERROR:critical error!"
/// -- No match: "critical_errors"
///
/// -- Empty string handling --
/// SELECT matches_term(column, '') FROM table;
/// -- Text: "" => true
/// -- Text: "any" => false
///
/// -- Case sensitivity --
/// SELECT matches_term(column, 'Cat') FROM table;
/// -- Text: "Cat" => true
/// -- Text: "cat" => false
/// ```
pub struct MatchesTermFunction;
impl MatchesTermFunction {
pub fn register(registry: &FunctionRegistry) {
registry.register(Arc::new(MatchesTermFunction));
}
}
impl fmt::Display for MatchesTermFunction {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
write!(f, "MATCHES_TERM")
}
}
impl Function for MatchesTermFunction {
fn name(&self) -> &str {
"matches_term"
}
fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result<ConcreteDataType> {
Ok(ConcreteDataType::boolean_datatype())
}
fn signature(&self) -> common_query::prelude::Signature {
common_query::prelude::Signature::exact(
vec![
ConcreteDataType::string_datatype(),
ConcreteDataType::string_datatype(),
],
Volatility::Immutable,
)
}
fn eval(&self, _func_ctx: &FunctionContext, columns: &[VectorRef]) -> Result<VectorRef> {
ensure!(
columns.len() == 2,
InvalidFuncArgsSnafu {
err_msg: format!(
"The length of the args is not correct, expect exactly 2, have: {}",
columns.len()
),
}
);
let text_column = &columns[0];
if text_column.is_empty() {
return Ok(Arc::new(BooleanVector::from(Vec::<bool>::with_capacity(0))));
}
let term_column = &columns[1];
let compiled_finder = if term_column.is_const() {
let term = term_column.get_ref(0).as_string().unwrap();
match term {
None => {
return Ok(Arc::new(BooleanVector::from_iter(
iter::repeat(None).take(text_column.len()),
)));
}
Some(term) => Some(MatchesTermFinder::new(term)),
}
} else {
None
};
let len = text_column.len();
let mut result = BooleanVectorBuilder::with_capacity(len);
for i in 0..len {
let text = text_column.get_ref(i).as_string().unwrap();
let Some(text) = text else {
result.push_null();
continue;
};
let contains = match &compiled_finder {
Some(finder) => finder.find(text),
None => {
let term = match term_column.get_ref(i).as_string().unwrap() {
None => {
result.push_null();
continue;
}
Some(term) => term,
};
MatchesTermFinder::new(term).find(text)
}
};
result.push(Some(contains));
}
Ok(result.to_vector())
}
}
/// A compiled finder for `matches_term` function that holds the compiled term
/// and its metadata for efficient matching.
///
/// A term is considered matched when:
/// 1. The exact sequence appears in the text
/// 2. It is either:
/// - At the start/end of text with adjacent non-alphanumeric character
/// - Surrounded by non-alphanumeric characters
///
/// # Examples
/// ```
/// let finder = MatchesTermFinder::new("cat");
/// assert!(finder.find("cat!")); // Term at end with punctuation
/// assert!(finder.find("dog,cat")); // Term preceded by comma
/// assert!(!finder.find("category")); // Partial match rejected
///
/// let finder = MatchesTermFinder::new("world");
/// assert!(finder.find("hello-world")); // Hyphen boundary
/// ```
#[derive(Clone, Debug)]
pub struct MatchesTermFinder {
finder: memmem::Finder<'static>,
term: String,
starts_with_non_alnum: bool,
ends_with_non_alnum: bool,
}
impl MatchesTermFinder {
/// Create a new `MatchesTermFinder` for the given term.
pub fn new(term: &str) -> Self {
let starts_with_non_alnum = term.chars().next().is_some_and(|c| !c.is_alphanumeric());
let ends_with_non_alnum = term.chars().last().is_some_and(|c| !c.is_alphanumeric());
Self {
finder: memmem::Finder::new(term).into_owned(),
term: term.to_string(),
starts_with_non_alnum,
ends_with_non_alnum,
}
}
/// Find the term in the text.
pub fn find(&self, text: &str) -> bool {
if self.term.is_empty() {
return text.is_empty();
}
if text.len() < self.term.len() {
return false;
}
let mut pos = 0;
while let Some(found_pos) = self.finder.find(text[pos..].as_bytes()) {
let actual_pos = pos + found_pos;
let prev_ok = self.starts_with_non_alnum
|| text[..actual_pos]
.chars()
.last()
.map(|c| !c.is_alphanumeric())
.unwrap_or(true);
if prev_ok {
let next_pos = actual_pos + self.finder.needle().len();
let next_ok = self.ends_with_non_alnum
|| text[next_pos..]
.chars()
.next()
.map(|c| !c.is_alphanumeric())
.unwrap_or(true);
if next_ok {
return true;
}
}
if let Some(next_char) = text[actual_pos..].chars().next() {
pos = actual_pos + next_char.len_utf8();
} else {
break;
}
}
false
}
}
#[cfg(test)]
mod tests {
use super::*;
#[test]
fn matches_term_example() {
let finder = MatchesTermFinder::new("hello world");
assert!(finder.find("warning:hello world!"));
assert!(!finder.find("hello-world"));
assert!(!finder.find("hello world2023"));
let finder = MatchesTermFinder::new("critical error");
assert!(finder.find("ERROR:critical error!"));
assert!(!finder.find("critical_errors"));
let finder = MatchesTermFinder::new("");
assert!(finder.find(""));
assert!(!finder.find("any"));
let finder = MatchesTermFinder::new("Cat");
assert!(finder.find("Cat"));
assert!(!finder.find("cat"));
}
#[test]
fn matches_term_with_punctuation() {
assert!(MatchesTermFinder::new("cat").find("cat!"));
assert!(MatchesTermFinder::new("dog").find("!dog"));
}
#[test]
fn matches_phrase_with_boundaries() {
assert!(MatchesTermFinder::new("hello-world").find("hello-world"));
assert!(MatchesTermFinder::new("'foo bar'").find("test: 'foo bar'"));
}
#[test]
fn matches_at_text_boundaries() {
assert!(MatchesTermFinder::new("start").find("start..."));
assert!(MatchesTermFinder::new("end").find("...end"));
}
// Negative cases
#[test]
fn rejects_partial_matches() {
assert!(!MatchesTermFinder::new("cat").find("category"));
assert!(!MatchesTermFinder::new("boot").find("rebooted"));
}
#[test]
fn rejects_missing_term() {
assert!(!MatchesTermFinder::new("foo").find("hello world"));
}
// Edge cases
#[test]
fn handles_empty_inputs() {
assert!(!MatchesTermFinder::new("test").find(""));
assert!(!MatchesTermFinder::new("").find("text"));
}
#[test]
fn different_unicode_boundaries() {
assert!(MatchesTermFinder::new("café").find("café>"));
assert!(!MatchesTermFinder::new("café").find("口café>"));
assert!(!MatchesTermFinder::new("café").find("café口"));
assert!(!MatchesTermFinder::new("café").find("cafémore"));
assert!(MatchesTermFinder::new("русский").find("русский!"));
assert!(MatchesTermFinder::new("русский").find("русский!"));
}
#[test]
fn case_sensitive_matching() {
assert!(!MatchesTermFinder::new("cat").find("Cat"));
assert!(MatchesTermFinder::new("CaT").find("CaT"));
}
#[test]
fn numbers_in_term() {
assert!(MatchesTermFinder::new("v1.0").find("v1.0!"));
assert!(!MatchesTermFinder::new("v1.0").find("v1.0a"));
}
#[test]
fn adjacent_alphanumeric_fails() {
assert!(!MatchesTermFinder::new("cat").find("cat5"));
assert!(!MatchesTermFinder::new("dog").find("dogcat"));
}
#[test]
fn empty_term_text() {
assert!(!MatchesTermFinder::new("").find("text"));
assert!(MatchesTermFinder::new("").find(""));
assert!(!MatchesTermFinder::new("text").find(""));
}
#[test]
fn leading_non_alphanumeric() {
assert!(MatchesTermFinder::new("/cat").find("dog/cat"));
assert!(MatchesTermFinder::new("dog/").find("dog/cat"));
assert!(MatchesTermFinder::new("dog/cat").find("dog/cat"));
}
#[test]
fn continues_searching_after_boundary_mismatch() {
assert!(!MatchesTermFinder::new("log").find("bloglog!"));
assert!(MatchesTermFinder::new("log").find("bloglog log"));
assert!(MatchesTermFinder::new("log").find("alogblog_log!"));
assert!(MatchesTermFinder::new("error").find("errorlog_error_case"));
assert!(MatchesTermFinder::new("test").find("atestbtestc_test_end"));
assert!(MatchesTermFinder::new("data").find("database_data_store"));
assert!(!MatchesTermFinder::new("data").find("database_datastore"));
assert!(MatchesTermFinder::new("log.txt").find("catalog.txt_log.txt!"));
assert!(!MatchesTermFinder::new("log.txt").find("catalog.txtlog.txt!"));
assert!(MatchesTermFinder::new("data-set").find("bigdata-set_data-set!"));
assert!(MatchesTermFinder::new("中文").find("这是中文测试,中文!"));
assert!(MatchesTermFinder::new("error").find("错误errorerror日志_error!"));
}
}

View File

@@ -22,7 +22,9 @@ mod version;
use std::sync::Arc;
use build::BuildFunction;
use database::{CurrentSchemaFunction, DatabaseFunction, SessionUserFunction};
use database::{
CurrentSchemaFunction, DatabaseFunction, ReadPreferenceFunction, SessionUserFunction,
};
use pg_catalog::PGCatalogFunction;
use procedure_state::ProcedureStateFunction;
use timezone::TimezoneFunction;
@@ -39,6 +41,7 @@ impl SystemFunction {
registry.register(Arc::new(CurrentSchemaFunction));
registry.register(Arc::new(DatabaseFunction));
registry.register(Arc::new(SessionUserFunction));
registry.register(Arc::new(ReadPreferenceFunction));
registry.register(Arc::new(TimezoneFunction));
registry.register_async(Arc::new(ProcedureStateFunction));
PGCatalogFunction::register(registry);

View File

@@ -30,9 +30,12 @@ pub struct DatabaseFunction;
pub struct CurrentSchemaFunction;
pub struct SessionUserFunction;
pub struct ReadPreferenceFunction;
const DATABASE_FUNCTION_NAME: &str = "database";
const CURRENT_SCHEMA_FUNCTION_NAME: &str = "current_schema";
const SESSION_USER_FUNCTION_NAME: &str = "session_user";
const READ_PREFERENCE_FUNCTION_NAME: &str = "read_preference";
impl Function for DatabaseFunction {
fn name(&self) -> &str {
@@ -94,6 +97,26 @@ impl Function for SessionUserFunction {
}
}
impl Function for ReadPreferenceFunction {
fn name(&self) -> &str {
READ_PREFERENCE_FUNCTION_NAME
}
fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result<ConcreteDataType> {
Ok(ConcreteDataType::string_datatype())
}
fn signature(&self) -> Signature {
Signature::nullary(Volatility::Immutable)
}
fn eval(&self, func_ctx: &FunctionContext, _columns: &[VectorRef]) -> Result<VectorRef> {
let read_preference = func_ctx.query_ctx.read_preference();
Ok(Arc::new(StringVector::from_slice(&[read_preference.as_ref()])) as _)
}
}
impl fmt::Display for DatabaseFunction {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
write!(f, "DATABASE")
@@ -112,6 +135,12 @@ impl fmt::Display for SessionUserFunction {
}
}
impl fmt::Display for ReadPreferenceFunction {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
write!(f, "READ_PREFERENCE")
}
}
#[cfg(test)]
mod tests {
use std::sync::Arc;

View File

@@ -15,11 +15,13 @@
use api::helper::ColumnDataTypeWrapper;
use api::v1::add_column_location::LocationType;
use api::v1::alter_table_expr::Kind;
use api::v1::column_def::{as_fulltext_option, as_skipping_index_type};
use api::v1::column_def::{
as_fulltext_option_analyzer, as_fulltext_option_backend, as_skipping_index_type,
};
use api::v1::{
column_def, AddColumnLocation as Location, AlterTableExpr, Analyzer, CreateTableExpr,
DropColumns, ModifyColumnTypes, RenameTable, SemanticType,
SkippingIndexType as PbSkippingIndexType,
DropColumns, FulltextBackend as PbFulltextBackend, ModifyColumnTypes, RenameTable,
SemanticType, SkippingIndexType as PbSkippingIndexType,
};
use common_query::AddColumnLocation;
use datatypes::schema::{ColumnSchema, FulltextOptions, RawSchema, SkippingIndexOptions};
@@ -126,11 +128,15 @@ pub fn alter_expr_to_request(table_id: TableId, expr: AlterTableExpr) -> Result<
column_name: f.column_name.clone(),
options: FulltextOptions {
enable: f.enable,
analyzer: as_fulltext_option(
analyzer: as_fulltext_option_analyzer(
Analyzer::try_from(f.analyzer)
.context(InvalidSetFulltextOptionRequestSnafu)?,
),
case_sensitive: f.case_sensitive,
backend: as_fulltext_option_backend(
PbFulltextBackend::try_from(f.backend)
.context(InvalidSetFulltextOptionRequestSnafu)?,
),
},
},
},

View File

@@ -92,6 +92,22 @@ pub struct RegionStat {
pub sst_size: u64,
/// The size of the SST index files in bytes.
pub index_size: u64,
/// The manifest infoof the region.
pub region_manifest: RegionManifestInfo,
}
#[derive(Debug, Clone, Copy, Serialize, Deserialize)]
pub enum RegionManifestInfo {
Mito {
manifest_version: u64,
flushed_entry_id: u64,
},
Metric {
data_manifest_version: u64,
data_flushed_entry_id: u64,
metadata_manifest_version: u64,
metadata_flushed_entry_id: u64,
},
}
impl Stat {
@@ -165,6 +181,31 @@ impl TryFrom<&HeartbeatRequest> for Stat {
}
}
impl From<store_api::region_engine::RegionManifestInfo> for RegionManifestInfo {
fn from(value: store_api::region_engine::RegionManifestInfo) -> Self {
match value {
store_api::region_engine::RegionManifestInfo::Mito {
manifest_version,
flushed_entry_id,
} => RegionManifestInfo::Mito {
manifest_version,
flushed_entry_id,
},
store_api::region_engine::RegionManifestInfo::Metric {
data_manifest_version,
data_flushed_entry_id,
metadata_manifest_version,
metadata_flushed_entry_id,
} => RegionManifestInfo::Metric {
data_manifest_version,
data_flushed_entry_id,
metadata_manifest_version,
metadata_flushed_entry_id,
},
}
}
}
impl From<&api::v1::meta::RegionStat> for RegionStat {
fn from(value: &api::v1::meta::RegionStat) -> Self {
let region_stat = value
@@ -185,6 +226,7 @@ impl From<&api::v1::meta::RegionStat> for RegionStat {
manifest_size: region_stat.manifest_size,
sst_size: region_stat.sst_size,
index_size: region_stat.index_size,
region_manifest: region_stat.manifest.into(),
}
}
}

View File

@@ -28,6 +28,7 @@ use crate::key::table_route::PhysicalTableRouteValue;
use crate::key::TableMetadataManagerRef;
use crate::node_manager::NodeManagerRef;
use crate::region_keeper::MemoryRegionKeeperRef;
use crate::region_registry::LeaderRegionRegistryRef;
use crate::rpc::ddl::{SubmitDdlTaskRequest, SubmitDdlTaskResponse};
use crate::rpc::procedure::{
AddRegionFollowerRequest, MigrateRegionRequest, MigrateRegionResponse, ProcedureStateResponse,
@@ -164,6 +165,8 @@ pub struct DdlContext {
pub cache_invalidator: CacheInvalidatorRef,
/// Keep tracking operating regions.
pub memory_region_keeper: MemoryRegionKeeperRef,
/// The leader region registry.
pub leader_region_registry: LeaderRegionRegistryRef,
/// Table metadata manager.
pub table_metadata_manager: TableMetadataManagerRef,
/// Allocator for table metadata.

View File

@@ -22,30 +22,31 @@ use std::vec;
use api::v1::alter_table_expr::Kind;
use api::v1::RenameTable;
use async_trait::async_trait;
use common_error::ext::ErrorExt;
use common_error::status_code::StatusCode;
use common_error::ext::BoxedError;
use common_procedure::error::{FromJsonSnafu, Result as ProcedureResult, ToJsonSnafu};
use common_procedure::{
Context as ProcedureContext, Error as ProcedureError, LockKey, Procedure, Status, StringKey,
Context as ProcedureContext, ContextProvider, Error as ProcedureError, LockKey, PoisonKey,
PoisonKeys, Procedure, ProcedureId, Status, StringKey,
};
use common_telemetry::{debug, error, info};
use futures::future;
use serde::{Deserialize, Serialize};
use snafu::ResultExt;
use snafu::{ensure, ResultExt};
use store_api::storage::RegionId;
use strum::AsRefStr;
use table::metadata::{RawTableInfo, TableId, TableInfo};
use table::table_reference::TableReference;
use crate::cache_invalidator::Context;
use crate::ddl::utils::add_peer_context_if_needed;
use crate::ddl::utils::{add_peer_context_if_needed, handle_multiple_results, MultipleResults};
use crate::ddl::DdlContext;
use crate::error::{Error, Result};
use crate::error::{AbortProcedureSnafu, Error, NoLeaderSnafu, PutPoisonSnafu, Result};
use crate::instruction::CacheIdent;
use crate::key::table_info::TableInfoValue;
use crate::key::{DeserializedValueWithBytes, RegionDistribution};
use crate::lock_key::{CatalogLock, SchemaLock, TableLock, TableNameLock};
use crate::metrics;
use crate::poison_key::table_poison_key;
use crate::rpc::ddl::AlterTableTask;
use crate::rpc::router::{find_leader_regions, find_leaders, region_distribution};
@@ -104,7 +105,27 @@ impl AlterTableProcedure {
Ok(Status::executing(true))
}
pub async fn submit_alter_region_requests(&mut self) -> Result<Status> {
fn table_poison_key(&self) -> PoisonKey {
table_poison_key(self.data.table_id())
}
async fn put_poison(
&self,
ctx_provider: &dyn ContextProvider,
procedure_id: ProcedureId,
) -> Result<()> {
let poison_key = self.table_poison_key();
ctx_provider
.try_put_poison(&poison_key, procedure_id)
.await
.context(PutPoisonSnafu)
}
pub async fn submit_alter_region_requests(
&mut self,
procedure_id: ProcedureId,
ctx_provider: &dyn ContextProvider,
) -> Result<Status> {
let table_id = self.data.table_id();
let (_, physical_table_route) = self
.context
@@ -127,6 +148,9 @@ impl AlterTableProcedure {
alter_kind,
);
ensure!(!leaders.is_empty(), NoLeaderSnafu { table_id });
// Puts the poison before submitting alter region requests to datanodes.
self.put_poison(ctx_provider, procedure_id).await?;
for datanode in leaders {
let requester = self.context.node_manager.datanode(&datanode).await;
let regions = find_leader_regions(&physical_table_route.region_routes, &datanode);
@@ -140,28 +164,51 @@ impl AlterTableProcedure {
let requester = requester.clone();
alter_region_tasks.push(async move {
if let Err(err) = requester.handle(request).await {
if err.status_code() != StatusCode::RequestOutdated {
// Treat request outdated as success.
// The engine will throw this code when the schema version not match.
// As this procedure has locked the table, the only reason for this error
// is procedure is succeeded before and is retrying.
return Err(add_peer_context_if_needed(datanode)(err));
}
}
Ok(())
requester
.handle(request)
.await
.map_err(add_peer_context_if_needed(datanode))
});
}
}
future::join_all(alter_region_tasks)
let results = future::join_all(alter_region_tasks)
.await
.into_iter()
.collect::<Result<Vec<_>>>()?;
.collect::<Vec<_>>();
self.data.state = AlterTableState::UpdateMetadata;
match handle_multiple_results(results) {
MultipleResults::PartialRetryable(error) => {
// Just returns the error, and wait for the next try.
Err(error)
}
MultipleResults::PartialNonRetryable(error) => {
error!(error; "Partial non-retryable errors occurred during alter table, table {}, table_id: {}", self.data.table_ref(), self.data.table_id());
// No retry will be done.
Ok(Status::poisoned(
Some(self.table_poison_key()),
ProcedureError::external(error),
))
}
MultipleResults::AllRetryable(error) => {
// Just returns the error, and wait for the next try.
Err(error)
}
MultipleResults::Ok => {
self.data.state = AlterTableState::UpdateMetadata;
Ok(Status::executing_with_clean_poisons(true))
}
MultipleResults::AllNonRetryable(error) => {
error!(error; "All alter requests returned non-retryable errors for table {}, table_id: {}", self.data.table_ref(), self.data.table_id());
// It assumes the metadata on datanode is not changed.
// Case: The alter region request is sent but not applied. (e.g., InvalidArgument)
Ok(Status::executing(true))
let err = BoxedError::new(error);
Err(err).context(AbortProcedureSnafu {
clean_poisons: true,
})
}
}
}
/// Update table metadata.
@@ -250,10 +297,12 @@ impl Procedure for AlterTableProcedure {
Self::TYPE_NAME
}
async fn execute(&mut self, _ctx: &ProcedureContext) -> ProcedureResult<Status> {
async fn execute(&mut self, ctx: &ProcedureContext) -> ProcedureResult<Status> {
let error_handler = |e: Error| {
if e.is_retry_later() {
ProcedureError::retry_later(e)
} else if e.need_clean_poisons() {
ProcedureError::external_and_clean_poisons(e)
} else {
ProcedureError::external(e)
}
@@ -269,7 +318,10 @@ impl Procedure for AlterTableProcedure {
match state {
AlterTableState::Prepare => self.on_prepare().await,
AlterTableState::SubmitAlterRegionRequests => self.submit_alter_region_requests().await,
AlterTableState::SubmitAlterRegionRequests => {
self.submit_alter_region_requests(ctx.procedure_id, ctx.provider.as_ref())
.await
}
AlterTableState::UpdateMetadata => self.on_update_metadata().await,
AlterTableState::InvalidateTableCache => self.on_broadcast().await,
}
@@ -285,6 +337,10 @@ impl Procedure for AlterTableProcedure {
LockKey::new(key)
}
fn poison_keys(&self) -> PoisonKeys {
PoisonKeys::new(vec![self.table_poison_key()])
}
}
#[derive(Debug, Serialize, Deserialize, AsRefStr)]

View File

@@ -299,7 +299,9 @@ impl Procedure for CreateTableProcedure {
.creator
.register_opening_regions(&self.context, &x.region_routes)
.map_err(BoxedError::new)
.context(ExternalSnafu)?;
.context(ExternalSnafu {
clean_poisons: false,
})?;
}
Ok(())

View File

@@ -130,7 +130,9 @@ impl Procedure for DropDatabaseProcedure {
self.state
.recover(&self.runtime_context)
.map_err(BoxedError::new)
.context(ExternalSnafu)
.context(ExternalSnafu {
clean_poisons: false,
})
}
async fn execute(&mut self, _ctx: &ProcedureContext) -> ProcedureResult<Status> {

View File

@@ -200,7 +200,9 @@ impl Procedure for DropTableProcedure {
if register_operating_regions {
self.register_dropping_regions()
.map_err(BoxedError::new)
.context(ExternalSnafu)?;
.context(ExternalSnafu {
clean_poisons: false,
})?;
}
Ok(())

View File

@@ -35,7 +35,9 @@ use crate::error::{self, Result};
use crate::instruction::CacheIdent;
use crate::key::table_name::TableNameKey;
use crate::key::table_route::TableRouteValue;
use crate::rpc::router::{find_leader_regions, find_leaders, RegionRoute};
use crate::rpc::router::{
find_leader_regions, find_leaders, operating_leader_regions, RegionRoute,
};
/// [Control] indicated to the caller whether to go to the next step.
#[derive(Debug)]
@@ -250,6 +252,11 @@ impl DropTableExecutor {
.into_iter()
.collect::<Result<Vec<_>>>()?;
// Deletes the leader region from registry.
let region_ids = operating_leader_regions(region_routes);
ctx.leader_region_registry
.batch_delete(region_ids.into_iter().map(|(region_id, _)| region_id));
Ok(())
}
}

View File

@@ -80,7 +80,13 @@ pub async fn create_logical_table(
let tasks = vec![test_create_logical_table_task(table_name)];
let mut procedure = CreateLogicalTablesProcedure::new(tasks, physical_table_id, ddl_context);
let status = procedure.on_prepare().await.unwrap();
assert_matches!(status, Status::Executing { persist: true });
assert_matches!(
status,
Status::Executing {
persist: true,
clean_poisons: false
}
);
let status = procedure.on_create_metadata().await.unwrap();
assert_matches!(status, Status::Done { .. });

View File

@@ -171,3 +171,74 @@ impl MockDatanodeHandler for NaiveDatanodeHandler {
unreachable!()
}
}
#[derive(Clone)]
pub struct PartialSuccessDatanodeHandler {
pub retryable: bool,
}
#[async_trait::async_trait]
impl MockDatanodeHandler for PartialSuccessDatanodeHandler {
async fn handle(&self, peer: &Peer, _request: RegionRequest) -> Result<RegionResponse> {
let success = peer.id % 2 == 0;
if success {
Ok(RegionResponse::new(0))
} else if self.retryable {
Err(Error::RetryLater {
source: BoxedError::new(
error::UnexpectedSnafu {
err_msg: "retry later",
}
.build(),
),
})
} else {
error::UnexpectedSnafu {
err_msg: "mock error",
}
.fail()
}
}
async fn handle_query(
&self,
_peer: &Peer,
_request: QueryRequest,
) -> Result<SendableRecordBatchStream> {
unreachable!()
}
}
#[derive(Clone)]
pub struct AllFailureDatanodeHandler {
pub retryable: bool,
}
#[async_trait::async_trait]
impl MockDatanodeHandler for AllFailureDatanodeHandler {
async fn handle(&self, _peer: &Peer, _request: RegionRequest) -> Result<RegionResponse> {
if self.retryable {
Err(Error::RetryLater {
source: BoxedError::new(
error::UnexpectedSnafu {
err_msg: "retry later",
}
.build(),
),
})
} else {
error::UnexpectedSnafu {
err_msg: "mock error",
}
.fail()
}
}
async fn handle_query(
&self,
_peer: &Peer,
_request: QueryRequest,
) -> Result<SendableRecordBatchStream> {
unreachable!()
}
}

View File

@@ -180,7 +180,13 @@ async fn test_on_prepare() {
let mut procedure = AlterLogicalTablesProcedure::new(tasks, phy_id, ddl_context);
let result = procedure.on_prepare().await;
assert_matches!(result, Ok(Status::Executing { persist: true }));
assert_matches!(
result,
Ok(Status::Executing {
persist: true,
clean_poisons: false
})
);
}
#[tokio::test]
@@ -205,7 +211,13 @@ async fn test_on_update_metadata() {
let mut procedure = AlterLogicalTablesProcedure::new(tasks, phy_id, ddl_context);
let mut status = procedure.on_prepare().await.unwrap();
assert_matches!(status, Status::Executing { persist: true });
assert_matches!(
status,
Status::Executing {
persist: true,
clean_poisons: false
}
);
let ctx = common_procedure::Context {
procedure_id: ProcedureId::random(),
@@ -213,10 +225,22 @@ async fn test_on_update_metadata() {
};
// on_submit_alter_region_requests
status = procedure.execute(&ctx).await.unwrap();
assert_matches!(status, Status::Executing { persist: true });
assert_matches!(
status,
Status::Executing {
persist: true,
clean_poisons: false
}
);
// on_update_metadata
status = procedure.execute(&ctx).await.unwrap();
assert_matches!(status, Status::Executing { persist: true });
assert_matches!(
status,
Status::Executing {
persist: true,
clean_poisons: false
}
);
}
#[tokio::test]
@@ -237,7 +261,13 @@ async fn test_on_part_duplicate_alter_request() {
let mut procedure = AlterLogicalTablesProcedure::new(tasks, phy_id, ddl_context.clone());
let mut status = procedure.on_prepare().await.unwrap();
assert_matches!(status, Status::Executing { persist: true });
assert_matches!(
status,
Status::Executing {
persist: true,
clean_poisons: false
}
);
let ctx = common_procedure::Context {
procedure_id: ProcedureId::random(),
@@ -245,10 +275,22 @@ async fn test_on_part_duplicate_alter_request() {
};
// on_submit_alter_region_requests
status = procedure.execute(&ctx).await.unwrap();
assert_matches!(status, Status::Executing { persist: true });
assert_matches!(
status,
Status::Executing {
persist: true,
clean_poisons: false
}
);
// on_update_metadata
status = procedure.execute(&ctx).await.unwrap();
assert_matches!(status, Status::Executing { persist: true });
assert_matches!(
status,
Status::Executing {
persist: true,
clean_poisons: false
}
);
// re-alter
let tasks = vec![
@@ -270,7 +312,13 @@ async fn test_on_part_duplicate_alter_request() {
let mut procedure = AlterLogicalTablesProcedure::new(tasks, phy_id, ddl_context.clone());
let mut status = procedure.on_prepare().await.unwrap();
assert_matches!(status, Status::Executing { persist: true });
assert_matches!(
status,
Status::Executing {
persist: true,
clean_poisons: false
}
);
let ctx = common_procedure::Context {
procedure_id: ProcedureId::random(),
@@ -278,10 +326,22 @@ async fn test_on_part_duplicate_alter_request() {
};
// on_submit_alter_region_requests
status = procedure.execute(&ctx).await.unwrap();
assert_matches!(status, Status::Executing { persist: true });
assert_matches!(
status,
Status::Executing {
persist: true,
clean_poisons: false
}
);
// on_update_metadata
status = procedure.execute(&ctx).await.unwrap();
assert_matches!(status, Status::Executing { persist: true });
assert_matches!(
status,
Status::Executing {
persist: true,
clean_poisons: false
}
);
let table_name_keys = vec![
TableNameKey::new(DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, "table1"),

View File

@@ -25,6 +25,9 @@ use api::v1::{
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
use common_error::ext::ErrorExt;
use common_error::status_code::StatusCode;
use common_procedure::store::poison_store::PoisonStore;
use common_procedure::{ProcedureId, Status};
use common_procedure_test::MockContextProvider;
use store_api::storage::RegionId;
use table::requests::TTL_KEY;
use tokio::sync::mpsc::{self};
@@ -33,16 +36,46 @@ use crate::ddl::alter_table::AlterTableProcedure;
use crate::ddl::test_util::alter_table::TestAlterTableExprBuilder;
use crate::ddl::test_util::create_table::test_create_table_task;
use crate::ddl::test_util::datanode_handler::{
DatanodeWatcher, RequestOutdatedErrorDatanodeHandler,
AllFailureDatanodeHandler, DatanodeWatcher, PartialSuccessDatanodeHandler,
RequestOutdatedErrorDatanodeHandler,
};
use crate::error::Error;
use crate::key::datanode_table::DatanodeTableKey;
use crate::key::table_name::TableNameKey;
use crate::key::table_route::TableRouteValue;
use crate::node_manager::NodeManagerRef;
use crate::peer::Peer;
use crate::poison_key::table_poison_key;
use crate::rpc::ddl::AlterTableTask;
use crate::rpc::router::{Region, RegionRoute};
use crate::test_util::{new_ddl_context, MockDatanodeManager};
fn prepare_table_route(table_id: u32) -> TableRouteValue {
TableRouteValue::physical(vec![
RegionRoute {
region: Region::new_test(RegionId::new(table_id, 1)),
leader_peer: Some(Peer::empty(1)),
follower_peers: vec![Peer::empty(5)],
leader_state: None,
leader_down_since: None,
},
RegionRoute {
region: Region::new_test(RegionId::new(table_id, 2)),
leader_peer: Some(Peer::empty(2)),
follower_peers: vec![Peer::empty(4)],
leader_state: None,
leader_down_since: None,
},
RegionRoute {
region: Region::new_test(RegionId::new(table_id, 3)),
leader_peer: Some(Peer::empty(3)),
follower_peers: vec![],
leader_state: None,
leader_down_since: None,
},
])
}
fn test_rename_alter_table_task(table_name: &str, new_table_name: &str) -> AlterTableTask {
let builder = TestAlterTableExprBuilder::default()
.table_name(table_name)
@@ -101,29 +134,7 @@ async fn test_on_submit_alter_request() {
.table_metadata_manager
.create_table_metadata(
task.table_info.clone(),
TableRouteValue::physical(vec![
RegionRoute {
region: Region::new_test(RegionId::new(table_id, 1)),
leader_peer: Some(Peer::empty(1)),
follower_peers: vec![Peer::empty(5)],
leader_state: None,
leader_down_since: None,
},
RegionRoute {
region: Region::new_test(RegionId::new(table_id, 2)),
leader_peer: Some(Peer::empty(2)),
follower_peers: vec![Peer::empty(4)],
leader_state: None,
leader_down_since: None,
},
RegionRoute {
region: Region::new_test(RegionId::new(table_id, 3)),
leader_peer: Some(Peer::empty(3)),
follower_peers: vec![],
leader_state: None,
leader_down_since: None,
},
]),
prepare_table_route(table_id),
HashMap::new(),
)
.await
@@ -141,9 +152,15 @@ async fn test_on_submit_alter_request() {
})),
},
};
let mut procedure = AlterTableProcedure::new(table_id, alter_table_task, ddl_context).unwrap();
let procedure_id = ProcedureId::random();
let provider = Arc::new(MockContextProvider::default());
let mut procedure =
AlterTableProcedure::new(table_id, alter_table_task, ddl_context.clone()).unwrap();
procedure.on_prepare().await.unwrap();
procedure.submit_alter_region_requests().await.unwrap();
procedure
.submit_alter_region_requests(procedure_id, provider.as_ref())
.await
.unwrap();
let check = |peer: Peer,
request: RegionRequest,
@@ -185,29 +202,7 @@ async fn test_on_submit_alter_request_with_outdated_request() {
.table_metadata_manager
.create_table_metadata(
task.table_info.clone(),
TableRouteValue::physical(vec![
RegionRoute {
region: Region::new_test(RegionId::new(table_id, 1)),
leader_peer: Some(Peer::empty(1)),
follower_peers: vec![Peer::empty(5)],
leader_state: None,
leader_down_since: None,
},
RegionRoute {
region: Region::new_test(RegionId::new(table_id, 2)),
leader_peer: Some(Peer::empty(2)),
follower_peers: vec![Peer::empty(4)],
leader_state: None,
leader_down_since: None,
},
RegionRoute {
region: Region::new_test(RegionId::new(table_id, 3)),
leader_peer: Some(Peer::empty(3)),
follower_peers: vec![],
leader_state: None,
leader_down_since: None,
},
]),
prepare_table_route(table_id),
HashMap::new(),
)
.await
@@ -225,9 +220,15 @@ async fn test_on_submit_alter_request_with_outdated_request() {
})),
},
};
let procedure_id = ProcedureId::random();
let provider = Arc::new(MockContextProvider::default());
let mut procedure = AlterTableProcedure::new(table_id, alter_table_task, ddl_context).unwrap();
procedure.on_prepare().await.unwrap();
procedure.submit_alter_region_requests().await.unwrap();
let err = procedure
.submit_alter_region_requests(procedure_id, provider.as_ref())
.await
.unwrap_err();
assert!(!err.is_retry_later());
}
#[tokio::test]
@@ -326,9 +327,14 @@ async fn test_on_update_metadata_add_columns() {
})),
},
};
let procedure_id = ProcedureId::random();
let provider = Arc::new(MockContextProvider::default());
let mut procedure = AlterTableProcedure::new(table_id, task, ddl_context.clone()).unwrap();
procedure.on_prepare().await.unwrap();
procedure.submit_alter_region_requests().await.unwrap();
procedure
.submit_alter_region_requests(procedure_id, provider.as_ref())
.await
.unwrap();
procedure.on_update_metadata().await.unwrap();
let table_info = ddl_context
@@ -387,9 +393,14 @@ async fn test_on_update_table_options() {
})),
},
};
let procedure_id = ProcedureId::random();
let provider = Arc::new(MockContextProvider::default());
let mut procedure = AlterTableProcedure::new(table_id, task, ddl_context.clone()).unwrap();
procedure.on_prepare().await.unwrap();
procedure.submit_alter_region_requests().await.unwrap();
procedure
.submit_alter_region_requests(procedure_id, provider.as_ref())
.await
.unwrap();
procedure.on_update_metadata().await.unwrap();
let table_info = ddl_context
@@ -417,3 +428,156 @@ async fn test_on_update_table_options() {
HashMap::from(&table_info.meta.options)
);
}
async fn prepare_alter_table_procedure(
node_manager: NodeManagerRef,
) -> (AlterTableProcedure, ProcedureId) {
common_telemetry::init_default_ut_logging();
let ddl_context = new_ddl_context(node_manager);
let table_id = 1024;
let table_name = "foo";
let task = test_create_table_task(table_name, table_id);
// Puts a value to table name key.
ddl_context
.table_metadata_manager
.create_table_metadata(
task.table_info.clone(),
prepare_table_route(table_id),
HashMap::new(),
)
.await
.unwrap();
let alter_table_task = AlterTableTask {
alter_table: AlterTableExpr {
catalog_name: DEFAULT_CATALOG_NAME.to_string(),
schema_name: DEFAULT_SCHEMA_NAME.to_string(),
table_name: table_name.to_string(),
kind: Some(Kind::DropColumns(DropColumns {
drop_columns: vec![DropColumn {
name: "cpu".to_string(),
}],
})),
},
};
let procedure_id = ProcedureId::random();
let mut procedure =
AlterTableProcedure::new(table_id, alter_table_task, ddl_context.clone()).unwrap();
procedure.on_prepare().await.unwrap();
(procedure, procedure_id)
}
#[tokio::test]
async fn test_on_submit_alter_request_with_partial_success_retryable() {
let node_manager = Arc::new(MockDatanodeManager::new(PartialSuccessDatanodeHandler {
retryable: true,
}));
let provider = Arc::new(MockContextProvider::default());
let (mut procedure, procedure_id) = prepare_alter_table_procedure(node_manager).await;
let result = procedure
.submit_alter_region_requests(procedure_id, provider.as_ref())
.await
.unwrap_err();
assert!(result.is_retry_later());
// Submits again
let result = procedure
.submit_alter_region_requests(procedure_id, provider.as_ref())
.await
.unwrap_err();
assert!(result.is_retry_later());
}
#[tokio::test]
async fn test_on_submit_alter_request_with_partial_success_non_retryable() {
let node_manager = Arc::new(MockDatanodeManager::new(PartialSuccessDatanodeHandler {
retryable: false,
}));
let provider = Arc::new(MockContextProvider::default());
let (mut procedure, procedure_id) = prepare_alter_table_procedure(node_manager).await;
let result = procedure
.submit_alter_region_requests(procedure_id, provider.as_ref())
.await
.unwrap();
assert_matches!(result, Status::Poisoned { .. });
// submits again
let result = procedure
.submit_alter_region_requests(procedure_id, provider.as_ref())
.await
.unwrap();
assert_matches!(result, Status::Poisoned { .. });
}
#[tokio::test]
async fn test_on_submit_alter_request_with_all_failure_retrybale() {
common_telemetry::init_default_ut_logging();
let node_manager = Arc::new(MockDatanodeManager::new(AllFailureDatanodeHandler {
retryable: true,
}));
let provider = Arc::new(MockContextProvider::default());
let (mut procedure, procedure_id) = prepare_alter_table_procedure(node_manager).await;
let err = procedure
.submit_alter_region_requests(procedure_id, provider.as_ref())
.await
.unwrap_err();
assert!(err.is_retry_later());
// submits again
let err = procedure
.submit_alter_region_requests(procedure_id, provider.as_ref())
.await
.unwrap_err();
assert!(err.is_retry_later());
}
#[tokio::test]
async fn test_on_submit_alter_request_with_all_failure_non_retrybale() {
common_telemetry::init_default_ut_logging();
let node_manager = Arc::new(MockDatanodeManager::new(AllFailureDatanodeHandler {
retryable: false,
}));
let provider = Arc::new(MockContextProvider::default());
let (mut procedure, procedure_id) = prepare_alter_table_procedure(node_manager).await;
let err = procedure
.submit_alter_region_requests(procedure_id, provider.as_ref())
.await
.unwrap_err();
assert_matches!(err, Error::AbortProcedure { .. });
assert!(!err.is_retry_later());
assert!(err.need_clean_poisons());
// submits again
let err = procedure
.submit_alter_region_requests(procedure_id, provider.as_ref())
.await
.unwrap_err();
assert_matches!(err, Error::AbortProcedure { .. });
assert!(!err.is_retry_later());
assert!(err.need_clean_poisons());
}
#[tokio::test]
async fn test_on_submit_alter_request_with_exist_poison() {
common_telemetry::init_default_ut_logging();
let node_manager = Arc::new(MockDatanodeManager::new(AllFailureDatanodeHandler {
retryable: false,
}));
let provider = Arc::new(MockContextProvider::default());
let (mut procedure, procedure_id) = prepare_alter_table_procedure(node_manager).await;
let table_id = 1024;
let key = table_poison_key(table_id).to_string();
let another_procedure_id = ProcedureId::random();
provider
.poison_manager()
.try_put_poison(key, another_procedure_id.to_string())
.await
.unwrap();
procedure.on_prepare().await.unwrap();
let err = procedure
.submit_alter_region_requests(procedure_id, provider.as_ref())
.await
.unwrap_err();
assert_matches!(err, Error::PutPoison { .. });
}

View File

@@ -69,7 +69,13 @@ async fn test_on_prepare() {
let physical_table_id = table_id;
let mut procedure = CreateLogicalTablesProcedure::new(tasks, physical_table_id, ddl_context);
let status = procedure.on_prepare().await.unwrap();
assert_matches!(status, Status::Executing { persist: true });
assert_matches!(
status,
Status::Executing {
persist: true,
clean_poisons: false
}
);
}
#[tokio::test]
@@ -202,7 +208,13 @@ async fn test_on_prepare_part_logical_tables_exist() {
ddl_context,
);
let status = procedure.on_prepare().await.unwrap();
assert_matches!(status, Status::Executing { persist: true });
assert_matches!(
status,
Status::Executing {
persist: true,
clean_poisons: false
}
);
}
#[tokio::test]
@@ -238,7 +250,13 @@ async fn test_on_create_metadata() {
ddl_context,
);
let status = procedure.on_prepare().await.unwrap();
assert_matches!(status, Status::Executing { persist: true });
assert_matches!(
status,
Status::Executing {
persist: true,
clean_poisons: false
}
);
let ctx = ProcedureContext {
procedure_id: ProcedureId::random(),
provider: Arc::new(MockContextProvider::default()),
@@ -294,7 +312,13 @@ async fn test_on_create_metadata_part_logical_tables_exist() {
ddl_context,
);
let status = procedure.on_prepare().await.unwrap();
assert_matches!(status, Status::Executing { persist: true });
assert_matches!(
status,
Status::Executing {
persist: true,
clean_poisons: false
}
);
let ctx = ProcedureContext {
procedure_id: ProcedureId::random(),
provider: Arc::new(MockContextProvider::default()),
@@ -339,7 +363,13 @@ async fn test_on_create_metadata_err() {
ddl_context.clone(),
);
let status = procedure.on_prepare().await.unwrap();
assert_matches!(status, Status::Executing { persist: true });
assert_matches!(
status,
Status::Executing {
persist: true,
clean_poisons: false
}
);
let ctx = ProcedureContext {
procedure_id: ProcedureId::random(),
provider: Arc::new(MockContextProvider::default()),

View File

@@ -137,7 +137,13 @@ async fn test_on_prepare_without_create_if_table_exists() {
task.create_table.create_if_not_exists = true;
let mut procedure = CreateTableProcedure::new(task, ddl_context);
let status = procedure.on_prepare().await.unwrap();
assert_matches!(status, Status::Executing { persist: true });
assert_matches!(
status,
Status::Executing {
persist: true,
clean_poisons: false
}
);
assert_eq!(procedure.table_id(), 1024);
}

View File

@@ -153,7 +153,13 @@ async fn test_on_prepare_without_create_if_table_exists() {
task.create_view.create_if_not_exists = true;
let mut procedure = CreateViewProcedure::new(task, ddl_context);
let status = procedure.on_prepare().await.unwrap();
assert_matches!(status, Status::Executing { persist: true });
assert_matches!(
status,
Status::Executing {
persist: true,
clean_poisons: false
}
);
assert_eq!(procedure.view_id(), 1024);
}

View File

@@ -13,10 +13,12 @@
// limitations under the License.
use std::collections::HashMap;
use std::fmt::Debug;
use common_catalog::consts::METRIC_ENGINE;
use common_error::ext::BoxedError;
use common_procedure::error::Error as ProcedureError;
use common_telemetry::{error, warn};
use common_wal::options::WalOptions;
use snafu::{ensure, OptionExt, ResultExt};
use store_api::metric_engine_consts::LOGICAL_TABLE_METADATA_KEY;
@@ -38,6 +40,7 @@ use crate::rpc::router::RegionRoute;
/// Adds [Peer] context if the error is unretryable.
pub fn add_peer_context_if_needed(datanode: Peer) -> impl FnOnce(Error) -> Error {
move |err| {
error!(err; "Failed to operate datanode, peer: {}", datanode);
if !err.is_retry_later() {
return Err::<(), BoxedError>(BoxedError::new(err))
.context(OperateDatanodeSnafu { peer: datanode })
@@ -182,6 +185,85 @@ pub fn extract_region_wal_options(
Ok(region_wal_options)
}
/// The result of multiple operations.
///
/// - Ok: all operations are successful.
/// - PartialRetryable: if any operation is retryable and without non retryable error, the result is retryable.
/// - PartialNonRetryable: if any operation is non retryable, the result is non retryable.
/// - AllRetryable: all operations are retryable.
/// - AllNonRetryable: all operations are not retryable.
pub enum MultipleResults {
Ok,
PartialRetryable(Error),
PartialNonRetryable(Error),
AllRetryable(Error),
AllNonRetryable(Error),
}
/// Handles the results of alter region requests.
///
/// For partial success, we need to check if the errors are retryable.
/// If all the errors are retryable, we return a retryable error.
/// Otherwise, we return the first error.
pub fn handle_multiple_results<T: Debug>(results: Vec<Result<T>>) -> MultipleResults {
if results.is_empty() {
return MultipleResults::Ok;
}
let num_results = results.len();
let mut retryable_results = Vec::new();
let mut non_retryable_results = Vec::new();
let mut ok_results = Vec::new();
for result in results {
match result {
Ok(_) => ok_results.push(result),
Err(err) => {
if err.is_retry_later() {
retryable_results.push(err);
} else {
non_retryable_results.push(err);
}
}
}
}
common_telemetry::debug!(
"retryable_results: {}, non_retryable_results: {}, ok_results: {}",
retryable_results.len(),
non_retryable_results.len(),
ok_results.len()
);
if retryable_results.len() == num_results {
return MultipleResults::AllRetryable(retryable_results.into_iter().next().unwrap());
} else if non_retryable_results.len() == num_results {
warn!("all non retryable results: {}", non_retryable_results.len());
for err in &non_retryable_results {
error!(err; "non retryable error");
}
return MultipleResults::AllNonRetryable(non_retryable_results.into_iter().next().unwrap());
} else if ok_results.len() == num_results {
return MultipleResults::Ok;
} else if !retryable_results.is_empty()
&& !ok_results.is_empty()
&& non_retryable_results.is_empty()
{
return MultipleResults::PartialRetryable(retryable_results.into_iter().next().unwrap());
}
warn!(
"partial non retryable results: {}, retryable results: {}, ok results: {}",
non_retryable_results.len(),
retryable_results.len(),
ok_results.len()
);
for err in &non_retryable_results {
error!(err; "non retryable error");
}
// non_retryable_results.len() > 0
MultipleResults::PartialNonRetryable(non_retryable_results.into_iter().next().unwrap())
}
#[cfg(test)]
mod tests {
use super::*;

View File

@@ -834,6 +834,7 @@ mod tests {
use std::sync::Arc;
use common_procedure::local::LocalManager;
use common_procedure::test_util::InMemoryPoisonStore;
use super::DdlManager;
use crate::cache_invalidator::DummyCacheInvalidator;
@@ -850,6 +851,7 @@ mod tests {
use crate::node_manager::{DatanodeRef, FlownodeRef, NodeManager};
use crate::peer::Peer;
use crate::region_keeper::MemoryRegionKeeper;
use crate::region_registry::LeaderRegionRegistry;
use crate::sequence::SequenceBuilder;
use crate::state_store::KvStateStore;
use crate::wal_options_allocator::WalOptionsAllocator;
@@ -882,7 +884,12 @@ mod tests {
));
let state_store = Arc::new(KvStateStore::new(kv_backend.clone()));
let procedure_manager = Arc::new(LocalManager::new(Default::default(), state_store));
let poison_manager = Arc::new(InMemoryPoisonStore::default());
let procedure_manager = Arc::new(LocalManager::new(
Default::default(),
state_store,
poison_manager,
));
let _ = DdlManager::try_new(
DdlContext {
@@ -893,6 +900,7 @@ mod tests {
flow_metadata_manager,
flow_metadata_allocator,
memory_region_keeper: Arc::new(MemoryRegionKeeper::default()),
leader_region_registry: Arc::new(LeaderRegionRegistry::default()),
region_failure_detector_controller: Arc::new(NoopRegionFailureDetectorControl),
},
procedure_manager.clone(),

View File

@@ -449,6 +449,14 @@ pub enum Error {
#[snafu(display("Retry later"))]
RetryLater { source: BoxedError },
#[snafu(display("Abort procedure"))]
AbortProcedure {
#[snafu(implicit)]
location: Location,
source: BoxedError,
clean_poisons: bool,
},
#[snafu(display(
"Failed to encode a wal options to json string, wal_options: {:?}",
wal_options
@@ -748,6 +756,33 @@ pub enum Error {
#[snafu(source)]
error: serde_json::Error,
},
#[snafu(display("No leader found for table_id: {}", table_id))]
NoLeader {
table_id: TableId,
#[snafu(implicit)]
location: Location,
},
#[snafu(display(
"Procedure poison key already exists with a different value, key: {}, value: {}",
key,
value
))]
ProcedurePoisonConflict {
key: String,
value: String,
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Failed to put poison, table metadata may be corrupted"))]
PutPoison {
#[snafu(implicit)]
location: Location,
#[snafu(source)]
source: common_procedure::error::Error,
},
}
pub type Result<T> = std::result::Result<T, Error>;
@@ -766,7 +801,8 @@ impl ErrorExt for Error {
| SerializeToJson { .. }
| DeserializeFromJson { .. } => StatusCode::Internal,
ValueNotExist { .. } => StatusCode::Unexpected,
NoLeader { .. } => StatusCode::TableUnavailable,
ValueNotExist { .. } | ProcedurePoisonConflict { .. } => StatusCode::Unexpected,
Unsupported { .. } => StatusCode::Unsupported,
@@ -837,7 +873,9 @@ impl ErrorExt for Error {
OperateDatanode { source, .. } => source.status_code(),
Table { source, .. } => source.status_code(),
RetryLater { source, .. } => source.status_code(),
AbortProcedure { source, .. } => source.status_code(),
ConvertAlterTableRequest { source, .. } => source.status_code(),
PutPoison { source, .. } => source.status_code(),
ParseProcedureId { .. }
| InvalidNumTopics { .. }
@@ -908,6 +946,11 @@ impl Error {
matches!(self, Error::RetryLater { .. })
}
/// Determine whether it needs to clean poisons.
pub fn need_clean_poisons(&self) -> bool {
matches!(self, Error::AbortProcedure { clean_poisons, .. } if *clean_poisons)
}
/// Returns true if the response exceeds the size limit.
pub fn is_exceeded_size_limit(&self) -> bool {
match self {

View File

@@ -192,6 +192,12 @@ pub struct DropFlow {
pub flownode_ids: Vec<FlownodeId>,
}
/// Flushes a batch of regions.
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)]
pub struct FlushRegions {
pub region_ids: Vec<RegionId>,
}
#[derive(Debug, Clone, Serialize, Deserialize, Display, PartialEq)]
pub enum Instruction {
/// Opens a region.
@@ -208,6 +214,8 @@ pub enum Instruction {
DowngradeRegion(DowngradeRegion),
/// Invalidates batch cache.
InvalidateCaches(Vec<CacheIdent>),
/// Flushes regions.
FlushRegion(FlushRegions),
}
/// The reply of [UpgradeRegion].

View File

@@ -57,7 +57,10 @@
//! - This key is mainly used in constructing the view in Datanode and Frontend.
//!
//! 12. Kafka topic key: `__topic_name/kafka/{topic_name}`
//! - The key is used to mark existing topics in kafka for WAL.
//! - The key is used to track existing topics in Kafka.
//! - The value is a [TopicNameValue](crate::key::topic_name::TopicNameValue) struct; it contains the `pruned_entry_id` which represents
//! the highest entry id that has been pruned from the remote WAL.
//! - When a region uses this topic, it should start replaying entries from `pruned_entry_id + 1` (minimum available entry id).
//!
//! 13. Topic name to region map key `__topic_region/{topic_name}/{region_id}`
//! - Mapping {topic_name} to {region_id}
@@ -137,6 +140,7 @@ use table::metadata::{RawTableInfo, TableId};
use table::table_name::TableName;
use table_info::{TableInfoKey, TableInfoManager, TableInfoValue};
use table_name::{TableNameKey, TableNameManager, TableNameValue};
use topic_name::TopicNameManager;
use topic_region::{TopicRegionKey, TopicRegionManager};
use view_info::{ViewInfoKey, ViewInfoManager, ViewInfoValue};
@@ -156,6 +160,7 @@ use crate::kv_backend::txn::{Txn, TxnOp};
use crate::kv_backend::KvBackendRef;
use crate::rpc::router::{region_distribution, LeaderState, RegionRoute};
use crate::rpc::store::BatchDeleteRequest;
use crate::state_store::PoisonValue;
use crate::DatanodeId;
pub const NAME_PATTERN: &str = r"[a-zA-Z_:-][a-zA-Z0-9_:\-\.@#]*";
@@ -308,6 +313,7 @@ pub struct TableMetadataManager {
schema_manager: SchemaManager,
table_route_manager: TableRouteManager,
tombstone_manager: TombstoneManager,
topic_name_manager: TopicNameManager,
topic_region_manager: TopicRegionManager,
kv_backend: KvBackendRef,
}
@@ -459,6 +465,7 @@ impl TableMetadataManager {
schema_manager: SchemaManager::new(kv_backend.clone()),
table_route_manager: TableRouteManager::new(kv_backend.clone()),
tombstone_manager: TombstoneManager::new(kv_backend.clone()),
topic_name_manager: TopicNameManager::new(kv_backend.clone()),
topic_region_manager: TopicRegionManager::new(kv_backend.clone()),
kv_backend,
}
@@ -512,6 +519,14 @@ impl TableMetadataManager {
&self.table_route_manager
}
pub fn topic_name_manager(&self) -> &TopicNameManager {
&self.topic_name_manager
}
pub fn topic_region_manager(&self) -> &TopicRegionManager {
&self.topic_region_manager
}
#[cfg(feature = "testing")]
pub fn kv_backend(&self) -> &KvBackendRef {
&self.kv_backend
@@ -1320,7 +1335,8 @@ impl_metadata_value! {
TableFlowValue,
NodeAddressValue,
SchemaNameValue,
FlowStateValue
FlowStateValue,
PoisonValue
}
impl_optional_metadata_value! {

View File

@@ -15,11 +15,14 @@
use std::fmt::{self, Display};
use serde::{Deserialize, Serialize};
use snafu::{OptionExt, ResultExt};
use snafu::{ensure, OptionExt, ResultExt};
use crate::error::{DecodeJsonSnafu, Error, InvalidMetadataSnafu, Result};
use crate::ensure_values;
use crate::error::{self, DecodeJsonSnafu, Error, InvalidMetadataSnafu, Result, UnexpectedSnafu};
use crate::key::txn_helper::TxnOpGetResponseSet;
use crate::key::{
MetadataKey, KAFKA_TOPIC_KEY_PATTERN, KAFKA_TOPIC_KEY_PREFIX, LEGACY_TOPIC_KEY_PREFIX,
DeserializedValueWithBytes, MetadataKey, MetadataValue, KAFKA_TOPIC_KEY_PATTERN,
KAFKA_TOPIC_KEY_PREFIX, LEGACY_TOPIC_KEY_PREFIX,
};
use crate::kv_backend::txn::{Txn, TxnOp};
use crate::kv_backend::KvBackendRef;
@@ -31,8 +34,32 @@ pub struct TopicNameKey<'a> {
pub topic: &'a str,
}
#[derive(Debug, Serialize, Deserialize)]
pub struct TopicNameValue;
/// The value associated with a topic name key.
///
/// The `pruned_entry_id` is the highest entry id that has been pruned from the remote WAL.
/// When a region uses this topic, it should start replaying entries from `pruned_entry_id + 1` (minimal available entry id).
#[derive(Debug, Serialize, Deserialize, Default, Clone)]
pub struct TopicNameValue {
pub pruned_entry_id: u64,
}
impl TopicNameValue {
pub fn new(pruned_entry_id: u64) -> Self {
Self { pruned_entry_id }
}
}
impl MetadataValue for TopicNameValue {
fn try_from_raw_value(raw_value: &[u8]) -> Result<Self> {
let value = serde_json::from_slice::<TopicNameValue>(raw_value).context(DecodeJsonSnafu)?;
Ok(value)
}
fn try_as_raw_value(&self) -> Result<Vec<u8>> {
let raw_value = serde_json::to_vec(self).context(DecodeJsonSnafu)?;
Ok(raw_value)
}
}
impl<'a> TopicNameKey<'a> {
pub fn new(topic: &'a str) -> Self {
@@ -114,13 +141,16 @@ impl TopicNameManager {
{
let topics =
serde_json::from_slice::<Vec<String>>(&kv.value).context(DecodeJsonSnafu)?;
let mut reqs = topics
.iter()
.map(|topic| {
let key = TopicNameKey::new(topic);
TxnOp::Put(key.to_bytes(), vec![])
})
.collect::<Vec<_>>();
let mut reqs = Vec::with_capacity(topics.len() + 1);
for topic in topics {
let topic_name_key = TopicNameKey::new(&topic);
let topic_name_value = TopicNameValue::new(0);
let put_req = TxnOp::Put(
topic_name_key.to_bytes(),
topic_name_value.try_as_raw_value()?,
);
reqs.push(put_req);
}
let delete_req = TxnOp::Delete(LEGACY_TOPIC_KEY_PREFIX.as_bytes().to_vec());
reqs.push(delete_req);
let txn = Txn::new().and_then(reqs);
@@ -129,7 +159,7 @@ impl TopicNameManager {
Ok(())
}
/// Range query for topics.
/// Range query for topics. Only the keys are returned.
/// Caution: this method returns keys as String instead of values of range query since the topics are stored in keys.
pub async fn range(&self) -> Result<Vec<String>> {
let prefix = TopicNameKey::range_start_key();
@@ -142,25 +172,72 @@ impl TopicNameManager {
.collect::<Result<Vec<String>>>()
}
/// Put topics into kvbackend.
/// Put topics into kvbackend. The value is set to 0 by default.
pub async fn batch_put(&self, topic_name_keys: Vec<TopicNameKey<'_>>) -> Result<()> {
let mut kvs = Vec::with_capacity(topic_name_keys.len());
let topic_name_value = TopicNameValue::new(0);
for topic_name_key in &topic_name_keys {
let kv = KeyValue {
key: topic_name_key.to_bytes(),
value: topic_name_value.clone().try_as_raw_value()?,
};
kvs.push(kv);
}
let req = BatchPutRequest {
kvs: topic_name_keys
.iter()
.map(|key| KeyValue {
key: key.to_bytes(),
value: vec![],
})
.collect(),
kvs,
prev_kv: false,
};
self.kv_backend.batch_put(req).await?;
Ok(())
}
/// Get value for a specific topic.
pub async fn get(
&self,
topic: &str,
) -> Result<Option<DeserializedValueWithBytes<TopicNameValue>>> {
let key = TopicNameKey::new(topic);
let raw_key = key.to_bytes();
self.kv_backend
.get(&raw_key)
.await?
.map(|x| DeserializedValueWithBytes::from_inner_slice(&x.value))
.transpose()
}
/// Update the topic name key and value in the kv backend.
pub async fn update(
&self,
topic: &str,
pruned_entry_id: u64,
prev: Option<DeserializedValueWithBytes<TopicNameValue>>,
) -> Result<()> {
let key = TopicNameKey::new(topic);
let raw_key = key.to_bytes();
let value = TopicNameValue::new(pruned_entry_id);
let new_raw_value = value.try_as_raw_value()?;
let raw_value = prev.map(|v| v.get_raw_bytes()).unwrap_or_default();
let txn = Txn::compare_and_put(raw_key.clone(), raw_value, new_raw_value.clone());
let mut r = self.kv_backend.txn(txn).await?;
if !r.succeeded {
let mut set = TxnOpGetResponseSet::from(&mut r.responses);
let raw_value = TxnOpGetResponseSet::filter(raw_key)(&mut set)
.context(UnexpectedSnafu {
err_msg: "Reads the empty topic name value in comparing operation while updating TopicNameValue",
})?;
let op_name = "updating TopicNameValue";
ensure_values!(raw_value, new_raw_value, op_name);
}
Ok(())
}
}
#[cfg(test)]
mod tests {
use std::assert_matches::assert_matches;
use std::sync::Arc;
use super::*;
@@ -204,7 +281,19 @@ mod tests {
let topics = manager.range().await.unwrap();
assert_eq!(topics, all_topics);
let topics = manager.range().await.unwrap();
assert_eq!(topics, all_topics);
for topic in &topics {
let value = manager.get(topic).await.unwrap().unwrap();
assert_eq!(value.pruned_entry_id, 0);
manager.update(topic, 1, Some(value.clone())).await.unwrap();
let new_value = manager.get(topic).await.unwrap().unwrap();
assert_eq!(new_value.pruned_entry_id, 1);
// Update twice, nothing changed
manager.update(topic, 1, Some(value.clone())).await.unwrap();
let new_value = manager.get(topic).await.unwrap().unwrap();
assert_eq!(new_value.pruned_entry_id, 1);
// Bad cas, emit error
let err = manager.update(topic, 3, Some(value)).await.unwrap_err();
assert_matches!(err, error::Error::Unexpected { .. });
}
}
}

View File

@@ -37,8 +37,10 @@ pub mod metrics;
pub mod node_expiry_listener;
pub mod node_manager;
pub mod peer;
pub mod poison_key;
pub mod range_stream;
pub mod region_keeper;
pub mod region_registry;
pub mod rpc;
pub mod sequence;
pub mod state_store;

View File

@@ -27,6 +27,7 @@ const TABLE_NAME_LOCK_PREFIX: &str = "__table_name_lock";
const FLOW_NAME_LOCK_PREFIX: &str = "__flow_name_lock";
const REGION_LOCK_PREFIX: &str = "__region_lock";
const FLOW_LOCK_PREFIX: &str = "__flow_lock";
const REMOTE_WAL_LOCK_PREFIX: &str = "__remote_wal_lock";
/// [CatalogLock] acquires the lock on the tenant level.
pub enum CatalogLock<'a> {
@@ -231,6 +232,31 @@ impl From<FlowLock> for StringKey {
}
}
/// [RemoteWalLock] acquires the lock on the remote wal topic level.
pub enum RemoteWalLock {
Read(String),
Write(String),
}
impl Display for RemoteWalLock {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
let key = match self {
RemoteWalLock::Read(s) => s,
RemoteWalLock::Write(s) => s,
};
write!(f, "{}/{}", REMOTE_WAL_LOCK_PREFIX, key)
}
}
impl From<RemoteWalLock> for StringKey {
fn from(value: RemoteWalLock) -> Self {
match value {
RemoteWalLock::Write(_) => StringKey::Exclusive(value.to_string()),
RemoteWalLock::Read(_) => StringKey::Share(value.to_string()),
}
}
}
#[cfg(test)]
mod tests {
use common_procedure::StringKey;
@@ -308,5 +334,16 @@ mod tests {
string_key,
StringKey::Exclusive(format!("{}/{}", FLOW_LOCK_PREFIX, flow_id))
);
// The remote wal lock
let string_key: StringKey = RemoteWalLock::Read("foo".to_string()).into();
assert_eq!(
string_key,
StringKey::Share(format!("{}/{}", REMOTE_WAL_LOCK_PREFIX, "foo"))
);
let string_key: StringKey = RemoteWalLock::Write("foo".to_string()).into();
assert_eq!(
string_key,
StringKey::Exclusive(format!("{}/{}", REMOTE_WAL_LOCK_PREFIX, "foo"))
);
}
}

View File

@@ -0,0 +1,22 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use common_procedure::PoisonKey;
use store_api::storage::TableId;
/// Returns the poison key for the table.
pub fn table_poison_key(table_id: TableId) -> PoisonKey {
let key = format!("table/{}", table_id);
PoisonKey::new(&key)
}

View File

@@ -0,0 +1,186 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use std::collections::hash_map::Entry;
use std::collections::HashMap;
use std::sync::{Arc, RwLock};
use common_telemetry::warn;
use store_api::storage::RegionId;
use crate::datanode::RegionManifestInfo;
/// Represents information about a leader region in the cluster.
/// Contains the datanode id where the leader is located,
/// and the current manifest version.
#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)]
pub struct LeaderRegion {
pub datanode_id: u64,
pub manifest: LeaderRegionManifestInfo,
}
#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)]
pub enum LeaderRegionManifestInfo {
Mito {
manifest_version: u64,
flushed_entry_id: u64,
},
Metric {
data_manifest_version: u64,
data_flushed_entry_id: u64,
metadata_manifest_version: u64,
metadata_flushed_entry_id: u64,
},
}
impl From<RegionManifestInfo> for LeaderRegionManifestInfo {
fn from(value: RegionManifestInfo) -> Self {
match value {
RegionManifestInfo::Mito {
manifest_version,
flushed_entry_id,
} => LeaderRegionManifestInfo::Mito {
manifest_version,
flushed_entry_id,
},
RegionManifestInfo::Metric {
data_manifest_version,
data_flushed_entry_id,
metadata_manifest_version,
metadata_flushed_entry_id,
} => LeaderRegionManifestInfo::Metric {
data_manifest_version,
data_flushed_entry_id,
metadata_manifest_version,
metadata_flushed_entry_id,
},
}
}
}
impl LeaderRegionManifestInfo {
/// Returns the manifest version of the leader region.
pub fn manifest_version(&self) -> u64 {
match self {
LeaderRegionManifestInfo::Mito {
manifest_version, ..
} => *manifest_version,
LeaderRegionManifestInfo::Metric {
data_manifest_version,
..
} => *data_manifest_version,
}
}
/// Returns the flushed entry id of the leader region.
pub fn flushed_entry_id(&self) -> u64 {
match self {
LeaderRegionManifestInfo::Mito {
flushed_entry_id, ..
} => *flushed_entry_id,
LeaderRegionManifestInfo::Metric {
data_flushed_entry_id,
..
} => *data_flushed_entry_id,
}
}
/// Returns the minimum flushed entry id of the leader region.
/// It is used to determine the minimum flushed entry id that can be pruned in remote wal.
pub fn min_flushed_entry_id(&self) -> u64 {
match self {
LeaderRegionManifestInfo::Mito {
flushed_entry_id, ..
} => *flushed_entry_id,
LeaderRegionManifestInfo::Metric {
data_flushed_entry_id,
metadata_flushed_entry_id,
..
} => (*data_flushed_entry_id).min(*metadata_flushed_entry_id),
}
}
}
pub type LeaderRegionRegistryRef = Arc<LeaderRegionRegistry>;
/// Registry that maintains a mapping of all leader regions in the cluster.
/// Tracks which datanode is hosting the leader for each region and the corresponding
/// manifest version.
#[derive(Default)]
pub struct LeaderRegionRegistry {
inner: RwLock<HashMap<RegionId, LeaderRegion>>,
}
impl LeaderRegionRegistry {
/// Creates a new empty leader region registry.
pub fn new() -> Self {
Self {
inner: RwLock::new(HashMap::new()),
}
}
/// Gets the leader region for the given region ids.
pub fn batch_get<I: Iterator<Item = RegionId>>(
&self,
region_ids: I,
) -> HashMap<RegionId, LeaderRegion> {
let inner = self.inner.read().unwrap();
region_ids
.into_iter()
.flat_map(|region_id| {
inner
.get(&region_id)
.map(|leader_region| (region_id, *leader_region))
})
.collect::<HashMap<_, _>>()
}
/// Puts the leader regions into the registry.
pub fn batch_put(&self, key_values: Vec<(RegionId, LeaderRegion)>) {
let mut inner = self.inner.write().unwrap();
for (region_id, leader_region) in key_values {
match inner.entry(region_id) {
Entry::Vacant(entry) => {
entry.insert(leader_region);
}
Entry::Occupied(mut entry) => {
let manifest_version = entry.get().manifest.manifest_version();
if manifest_version > leader_region.manifest.manifest_version() {
warn!(
"Received a leader region with a smaller manifest version than the existing one, ignore it. region: {}, existing_manifest_version: {}, new_manifest_version: {}",
region_id,
manifest_version,
leader_region.manifest.manifest_version()
);
} else {
entry.insert(leader_region);
}
}
}
}
}
pub fn batch_delete<I: Iterator<Item = RegionId>>(&self, region_ids: I) {
let mut inner = self.inner.write().unwrap();
for region_id in region_ids {
inner.remove(&region_id);
}
}
/// Resets the registry to an empty state.
pub fn reset(&self) {
let mut inner = self.inner.write().unwrap();
inner.clear();
}
}

View File

@@ -20,6 +20,7 @@ use api::v1::meta::{
ProcedureMeta as PbProcedureMeta, ProcedureStateResponse as PbProcedureStateResponse,
ProcedureStatus as PbProcedureStatus,
};
use common_error::ext::ErrorExt;
use common_procedure::{ProcedureId, ProcedureInfo, ProcedureState};
use snafu::ResultExt;
@@ -73,14 +74,15 @@ pub fn procedure_state_to_pb_state(state: &ProcedureState) -> (PbProcedureStatus
match state {
ProcedureState::Running => (PbProcedureStatus::Running, String::default()),
ProcedureState::Done { .. } => (PbProcedureStatus::Done, String::default()),
ProcedureState::Retrying { error } => (PbProcedureStatus::Retrying, error.to_string()),
ProcedureState::Failed { error } => (PbProcedureStatus::Failed, error.to_string()),
ProcedureState::Retrying { error } => (PbProcedureStatus::Retrying, error.output_msg()),
ProcedureState::Failed { error } => (PbProcedureStatus::Failed, error.output_msg()),
ProcedureState::PrepareRollback { error } => {
(PbProcedureStatus::PrepareRollback, error.to_string())
(PbProcedureStatus::PrepareRollback, error.output_msg())
}
ProcedureState::RollingBack { error } => {
(PbProcedureStatus::RollingBack, error.to_string())
(PbProcedureStatus::RollingBack, error.output_msg())
}
ProcedureState::Poisoned { error, .. } => (PbProcedureStatus::Poisoned, error.output_msg()),
}
}

View File

@@ -14,16 +14,23 @@
use async_trait::async_trait;
use common_error::ext::BoxedError;
use common_procedure::error::{DeleteStatesSnafu, ListStateSnafu, PutStateSnafu};
use common_procedure::error::{
DeletePoisonSnafu, DeleteStatesSnafu, GetPoisonSnafu, ListStateSnafu, PutPoisonSnafu,
PutStateSnafu, Result as ProcedureResult,
};
use common_procedure::store::poison_store::PoisonStore;
use common_procedure::store::state_store::{KeySet, KeyValueStream, StateStore};
use common_procedure::store::util::multiple_value_stream;
use common_procedure::Result as ProcedureResult;
use futures::future::try_join_all;
use futures::StreamExt;
use itertools::Itertools;
use snafu::ResultExt;
use serde::{Deserialize, Serialize};
use snafu::{ensure, OptionExt, ResultExt};
use crate::error::Result;
use crate::error::{ProcedurePoisonConflictSnafu, Result, UnexpectedSnafu};
use crate::key::txn_helper::TxnOpGetResponseSet;
use crate::key::{DeserializedValueWithBytes, MetadataValue};
use crate::kv_backend::txn::{Compare, CompareOp, Txn, TxnOp};
use crate::kv_backend::KvBackendRef;
use crate::range_stream::PaginationStream;
use crate::rpc::store::{BatchDeleteRequest, PutRequest, RangeRequest};
@@ -32,11 +39,16 @@ use crate::rpc::KeyValue;
const DELIMITER: &str = "/";
const PROCEDURE_PREFIX: &str = "/__procedure__/";
const PROCEDURE_POISON_KEY_PREFIX: &str = "/__procedure_poison/";
fn with_prefix(key: &str) -> String {
format!("{PROCEDURE_PREFIX}{key}")
}
fn with_poison_prefix(key: &str) -> String {
format!("{}{}", PROCEDURE_POISON_KEY_PREFIX, key)
}
fn strip_prefix(key: &str) -> String {
key.trim_start_matches(PROCEDURE_PREFIX).to_string()
}
@@ -207,8 +219,168 @@ impl StateStore for KvStateStore {
}
}
/// The value of the poison key.
///
/// Each poison value contains a unique token that identifies the procedure.
/// While multiple procedures may use the same poison key (representing the same resource),
/// each procedure will have a distinct token value to differentiate its ownership.
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct PoisonValue {
token: String,
}
type PoisonDecodeResult = Result<Option<DeserializedValueWithBytes<PoisonValue>>>;
impl KvStateStore {
/// Builds a create poison transaction,
/// it expected the `__procedure_poison/{key}` wasn't occupied.
fn build_create_poison_txn(
&self,
key: &str,
value: &PoisonValue,
) -> Result<(
Txn,
impl FnOnce(&mut TxnOpGetResponseSet) -> PoisonDecodeResult,
)> {
let key = key.as_bytes().to_vec();
let value = value.try_as_raw_value()?;
let txn = Txn::put_if_not_exists(key.clone(), value);
Ok((
txn,
TxnOpGetResponseSet::decode_with(TxnOpGetResponseSet::filter(key)),
))
}
/// Builds a delete poison transaction,
/// it expected the `__procedure_poison/{key}` was occupied.
fn build_delete_poison_txn(
&self,
key: &str,
value: PoisonValue,
) -> Result<(
Txn,
impl FnOnce(&mut TxnOpGetResponseSet) -> PoisonDecodeResult,
)> {
let key = key.as_bytes().to_vec();
let value = value.try_as_raw_value()?;
let txn = Txn::new()
.when(vec![Compare::with_value(
key.clone(),
CompareOp::Equal,
value,
)])
.and_then(vec![TxnOp::Delete(key.clone())])
.or_else(vec![TxnOp::Get(key.clone())]);
Ok((
txn,
TxnOpGetResponseSet::decode_with(TxnOpGetResponseSet::filter(key)),
))
}
async fn get_poison_inner(&self, key: &str) -> Result<Option<PoisonValue>> {
let key = with_poison_prefix(key);
let value = self.kv_backend.get(key.as_bytes()).await?;
value
.map(|v| PoisonValue::try_from_raw_value(&v.value))
.transpose()
}
/// Put the poison.
///
/// If the poison is already put by other procedure, it will return an error.
async fn set_poison_inner(&self, key: &str, token: &str) -> Result<()> {
let key = with_poison_prefix(key);
let (txn, on_failure) = self.build_create_poison_txn(
&key,
&PoisonValue {
token: token.to_string(),
},
)?;
let mut resp = self.kv_backend.txn(txn).await?;
if !resp.succeeded {
let mut set = TxnOpGetResponseSet::from(&mut resp.responses);
let remote_value = on_failure(&mut set)?
.context(UnexpectedSnafu {
err_msg: "Reads the empty poison value in comparing operation of the put consistency poison",
})?
.into_inner();
ensure!(
remote_value.token == token,
ProcedurePoisonConflictSnafu {
key: &key,
value: &remote_value.token,
}
);
}
Ok(())
}
/// Deletes the poison.
///
/// If the poison is not put by the procedure, it will return an error.
async fn delete_poison_inner(&self, key: &str, token: &str) -> Result<()> {
let key = with_poison_prefix(key);
let (txn, on_failure) = self.build_delete_poison_txn(
&key,
PoisonValue {
token: token.to_string(),
},
)?;
let mut resp = self.kv_backend.txn(txn).await?;
if !resp.succeeded {
let mut set = TxnOpGetResponseSet::from(&mut resp.responses);
let remote_value = on_failure(&mut set)?;
ensure!(
remote_value.is_none(),
ProcedurePoisonConflictSnafu {
key: &key,
value: &remote_value.unwrap().into_inner().token,
}
);
}
Ok(())
}
}
#[async_trait]
impl PoisonStore for KvStateStore {
async fn try_put_poison(&self, key: String, token: String) -> ProcedureResult<()> {
self.set_poison_inner(&key, &token)
.await
.map_err(BoxedError::new)
.context(PutPoisonSnafu { key, token })
}
async fn delete_poison(&self, key: String, token: String) -> ProcedureResult<()> {
self.delete_poison_inner(&key, &token)
.await
.map_err(BoxedError::new)
.context(DeletePoisonSnafu { key, token })
}
async fn get_poison(&self, key: &str) -> ProcedureResult<Option<String>> {
self.get_poison_inner(key)
.await
.map(|v| v.map(|v| v.token))
.map_err(BoxedError::new)
.context(GetPoisonSnafu { key })
}
}
#[cfg(test)]
mod tests {
use std::assert_matches::assert_matches;
use std::env;
use std::sync::Arc;
@@ -219,6 +391,7 @@ mod tests {
use uuid::Uuid;
use super::*;
use crate::error::Error;
use crate::kv_backend::chroot::ChrootKvBackend;
use crate::kv_backend::etcd::EtcdStore;
use crate::kv_backend::memory::MemoryKvBackend;
@@ -397,4 +570,73 @@ mod tests {
)
.await;
}
#[tokio::test]
async fn test_poison() {
let mem_kv = Arc::new(MemoryKvBackend::default());
let poison_manager = KvStateStore::new(mem_kv.clone());
let key = "table/1";
let token = "expected_token";
poison_manager.set_poison_inner(key, token).await.unwrap();
// Put again, should be ok.
poison_manager.set_poison_inner(key, token).await.unwrap();
// Delete, should be ok.
poison_manager
.delete_poison_inner(key, token)
.await
.unwrap();
// Delete again, should be ok.
poison_manager
.delete_poison_inner(key, token)
.await
.unwrap();
}
#[tokio::test]
async fn test_consistency_poison_failed() {
let mem_kv = Arc::new(MemoryKvBackend::default());
let poison_manager = KvStateStore::new(mem_kv.clone());
let key = "table/1";
let token = "expected_token";
let token2 = "expected_token2";
poison_manager.set_poison_inner(key, token).await.unwrap();
let err = poison_manager
.set_poison_inner(key, token2)
.await
.unwrap_err();
assert_matches!(err, Error::ProcedurePoisonConflict { .. });
let err = poison_manager
.delete_poison_inner(key, token2)
.await
.unwrap_err();
assert_matches!(err, Error::ProcedurePoisonConflict { .. });
}
#[test]
fn test_serialize_deserialize() {
let key = "table/1";
let value = PoisonValue {
token: "expected_token".to_string(),
};
let serialized_key = with_poison_prefix(key).as_bytes().to_vec();
let serialized_value = value.try_as_raw_value().unwrap();
let expected_key = "/__procedure_poison/table/1";
let expected_value = r#"{"token":"expected_token"}"#;
assert_eq!(expected_key.as_bytes(), serialized_key);
assert_eq!(expected_value.as_bytes(), serialized_value);
}
}

View File

@@ -35,6 +35,7 @@ use crate::node_manager::{
};
use crate::peer::{Peer, PeerLookupService};
use crate::region_keeper::MemoryRegionKeeper;
use crate::region_registry::LeaderRegionRegistry;
use crate::sequence::SequenceBuilder;
use crate::wal_options_allocator::WalOptionsAllocator;
use crate::{DatanodeId, FlownodeId};
@@ -177,6 +178,7 @@ pub fn new_ddl_context_with_kv_backend(
node_manager,
cache_invalidator: Arc::new(DummyCacheInvalidator),
memory_region_keeper: Arc::new(MemoryRegionKeeper::new()),
leader_region_registry: Arc::new(LeaderRegionRegistry::default()),
table_metadata_allocator,
table_metadata_manager,
flow_metadata_allocator,

View File

@@ -30,7 +30,7 @@ use crate::error::{EncodeWalOptionsSnafu, InvalidTopicNamePrefixSnafu, Result};
use crate::key::NAME_PATTERN_REGEX;
use crate::kv_backend::KvBackendRef;
use crate::leadership_notifier::LeadershipChangeListener;
use crate::wal_options_allocator::topic_creator::build_kafka_topic_creator;
pub use crate::wal_options_allocator::topic_creator::build_kafka_topic_creator;
use crate::wal_options_allocator::topic_pool::KafkaTopicPool;
/// Allocates wal options in region granularity.

View File

@@ -12,14 +12,16 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::sync::Arc;
use common_telemetry::{error, info};
use common_wal::config::kafka::common::DEFAULT_BACKOFF_CONFIG;
use common_wal::config::kafka::MetasrvKafkaConfig;
use rskafka::client::error::Error as RsKafkaError;
use rskafka::client::error::ProtocolError::TopicAlreadyExists;
use rskafka::client::partition::{Compression, UnknownTopicHandling};
use rskafka::client::{Client, ClientBuilder};
use rskafka::record::Record;
use rskafka::BackoffConfig;
use snafu::ResultExt;
use crate::error::{
@@ -32,9 +34,11 @@ use crate::error::{
// The `DEFAULT_PARTITION` refers to the index of the partition.
const DEFAULT_PARTITION: i32 = 0;
type KafkaClientRef = Arc<Client>;
/// Creates topics in kafka.
pub struct KafkaTopicCreator {
client: Client,
client: KafkaClientRef,
/// The number of partitions per topic.
num_partitions: i32,
/// The replication factor of each topic.
@@ -44,6 +48,10 @@ pub struct KafkaTopicCreator {
}
impl KafkaTopicCreator {
pub fn client(&self) -> &KafkaClientRef {
&self.client
}
async fn create_topic(&self, topic: &String, client: &Client) -> Result<()> {
let controller = client
.controller_client()
@@ -127,16 +135,10 @@ impl KafkaTopicCreator {
pub async fn build_kafka_topic_creator(config: &MetasrvKafkaConfig) -> Result<KafkaTopicCreator> {
// Builds an kafka controller client for creating topics.
let backoff_config = BackoffConfig {
init_backoff: config.backoff.init,
max_backoff: config.backoff.max,
base: config.backoff.base as f64,
deadline: config.backoff.deadline,
};
let broker_endpoints = common_wal::resolve_to_ipv4(&config.connection.broker_endpoints)
.await
.context(ResolveKafkaEndpointSnafu)?;
let mut builder = ClientBuilder::new(broker_endpoints).backoff_config(backoff_config);
let mut builder = ClientBuilder::new(broker_endpoints).backoff_config(DEFAULT_BACKOFF_CONFIG);
if let Some(sasl) = &config.connection.sasl {
builder = builder.sasl_config(sasl.config.clone().into_sasl_config());
};
@@ -151,7 +153,7 @@ pub async fn build_kafka_topic_creator(config: &MetasrvKafkaConfig) -> Result<Ka
})?;
Ok(KafkaTopicCreator {
client,
client: Arc::new(client),
num_partitions: config.kafka_topic.num_partitions,
replication_factor: config.kafka_topic.replication_factor,
create_topic_timeout: config.kafka_topic.create_topic_timeout.as_millis() as i32,

View File

@@ -9,4 +9,5 @@ workspace = true
[dependencies]
async-trait.workspace = true
common-procedure.workspace = true
common-procedure = { workspace = true, features = ["testing"] }
snafu.workspace = true

View File

@@ -18,21 +18,32 @@ use std::collections::HashMap;
use std::sync::Arc;
use async_trait::async_trait;
use common_procedure::store::poison_store::PoisonStore;
use common_procedure::test_util::InMemoryPoisonStore;
use common_procedure::{
Context, ContextProvider, Output, Procedure, ProcedureId, ProcedureState, ProcedureWithId,
Result, Status,
Context, ContextProvider, Output, PoisonKey, Procedure, ProcedureId, ProcedureState,
ProcedureWithId, Result, Status,
};
/// A Mock [ContextProvider].
#[derive(Default)]
pub struct MockContextProvider {
states: HashMap<ProcedureId, ProcedureState>,
poison_manager: InMemoryPoisonStore,
}
impl MockContextProvider {
/// Returns a new provider.
pub fn new(states: HashMap<ProcedureId, ProcedureState>) -> MockContextProvider {
MockContextProvider { states }
MockContextProvider {
states,
poison_manager: InMemoryPoisonStore::default(),
}
}
/// Returns a reference to the poison manager.
pub fn poison_manager(&self) -> &InMemoryPoisonStore {
&self.poison_manager
}
}
@@ -41,6 +52,12 @@ impl ContextProvider for MockContextProvider {
async fn procedure_state(&self, procedure_id: ProcedureId) -> Result<Option<ProcedureState>> {
Ok(self.states.get(&procedure_id).cloned())
}
async fn try_put_poison(&self, key: &PoisonKey, procedure_id: ProcedureId) -> Result<()> {
self.poison_manager
.try_put_poison(key.to_string(), procedure_id.to_string())
.await
}
}
/// Executes a procedure until it returns [Status::Done].
@@ -61,6 +78,7 @@ pub async fn execute_procedure_until_done(procedure: &mut dyn Procedure) -> Opti
"Executing subprocedure is unsupported"
),
Status::Done { output } => return output,
Status::Poisoned { .. } => return None,
}
}
}
@@ -88,6 +106,7 @@ pub async fn execute_procedure_once(
false
}
Status::Done { .. } => true,
Status::Poisoned { .. } => false,
}
}
@@ -109,6 +128,7 @@ pub async fn execute_until_suspended_or_done(
Status::Executing { .. } => (),
Status::Suspended { subprocedures, .. } => return Some(subprocedures),
Status::Done { .. } => break,
Status::Poisoned { .. } => unreachable!(),
}
}

View File

@@ -21,14 +21,21 @@ use common_macro::stack_trace_debug;
use snafu::{Location, Snafu};
use crate::procedure::ProcedureId;
use crate::PoisonKey;
/// Procedure error.
#[derive(Snafu)]
#[snafu(visibility(pub))]
#[stack_trace_debug]
pub enum Error {
#[snafu(display("Failed to execute procedure due to external error"))]
External { source: BoxedError },
#[snafu(display(
"Failed to execute procedure due to external error, clean poisons: {}",
clean_poisons
))]
External {
source: BoxedError,
clean_poisons: bool,
},
#[snafu(display("Loader {} is already registered", name))]
LoaderConflict {
@@ -58,6 +65,13 @@ pub enum Error {
location: Location,
},
#[snafu(display("Too many running procedures, max: {}", max_running_procedures))]
TooManyRunningProcedures {
max_running_procedures: usize,
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Failed to put state, key: '{key}'"))]
PutState {
key: String,
@@ -66,6 +80,32 @@ pub enum Error {
source: BoxedError,
},
#[snafu(display("Failed to put poison, key: '{key}', token: '{token}'"))]
PutPoison {
key: String,
token: String,
#[snafu(implicit)]
location: Location,
source: BoxedError,
},
#[snafu(display("Failed to get poison, key: '{key}'"))]
GetPoison {
key: String,
#[snafu(implicit)]
location: Location,
source: BoxedError,
},
#[snafu(display("Failed to delete poison, key: '{key}', token: '{token}'"))]
DeletePoison {
key: String,
token: String,
#[snafu(implicit)]
location: Location,
source: BoxedError,
},
#[snafu(display("Failed to delete {}", key))]
DeleteState {
key: String,
@@ -175,6 +215,21 @@ pub enum Error {
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Procedure not found, procedure_id: {}", procedure_id))]
ProcedureNotFound {
procedure_id: ProcedureId,
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Poison key not defined, key: '{key}', procedure_id: '{procedure_id}'"))]
PoisonKeyNotDefined {
key: PoisonKey,
procedure_id: ProcedureId,
#[snafu(implicit)]
location: Location,
},
}
pub type Result<T> = std::result::Result<T, Error>;
@@ -185,14 +240,19 @@ impl ErrorExt for Error {
Error::External { source, .. }
| Error::PutState { source, .. }
| Error::DeleteStates { source, .. }
| Error::ListState { source, .. } => source.status_code(),
| Error::ListState { source, .. }
| Error::PutPoison { source, .. }
| Error::DeletePoison { source, .. }
| Error::GetPoison { source, .. } => source.status_code(),
Error::ToJson { .. }
| Error::DeleteState { .. }
| Error::FromJson { .. }
| Error::WaitWatcher { .. }
| Error::RetryLater { .. }
| Error::RollbackProcedureRecovered { .. } => StatusCode::Internal,
| Error::RollbackProcedureRecovered { .. }
| Error::TooManyRunningProcedures { .. }
| Error::PoisonKeyNotDefined { .. } => StatusCode::Internal,
Error::RetryTimesExceeded { .. }
| Error::RollbackTimesExceeded { .. }
@@ -204,7 +264,8 @@ impl ErrorExt for Error {
}
Error::ProcedurePanic { .. }
| Error::ParseSegmentKey { .. }
| Error::Unexpected { .. } => StatusCode::Unexpected,
| Error::Unexpected { .. }
| &Error::ProcedureNotFound { .. } => StatusCode::Unexpected,
Error::ProcedureExec { source, .. } => source.status_code(),
Error::StartRemoveOutdatedMetaTask { source, .. }
| Error::StopRemoveOutdatedMetaTask { source, .. } => source.status_code(),
@@ -221,6 +282,15 @@ impl Error {
pub fn external<E: ErrorExt + Send + Sync + 'static>(err: E) -> Error {
Error::External {
source: BoxedError::new(err),
clean_poisons: false,
}
}
/// Creates a new [Error::External] error from source `err` and clean poisons.
pub fn external_and_clean_poisons<E: ErrorExt + Send + Sync + 'static>(err: E) -> Error {
Error::External {
source: BoxedError::new(err),
clean_poisons: true,
}
}
@@ -236,6 +306,11 @@ impl Error {
matches!(self, Error::RetryLater { .. })
}
/// Determine whether it needs to clean poisons.
pub fn need_clean_poisons(&self) -> bool {
matches!(self, Error::External { clean_poisons, .. } if *clean_poisons)
}
/// Creates a new [Error::RetryLater] or [Error::External] error from source `err` according
/// to its [StatusCode].
pub fn from_error_ext<E: ErrorExt + Send + Sync + 'static>(err: E) -> Self {

View File

@@ -23,10 +23,13 @@ mod procedure;
pub mod store;
pub mod watcher;
#[cfg(any(test, feature = "testing"))]
pub mod test_util;
pub use crate::error::{Error, Result};
pub use crate::procedure::{
BoxedProcedure, BoxedProcedureLoader, Context, ContextProvider, LockKey, Output, ParseIdError,
Procedure, ProcedureId, ProcedureInfo, ProcedureManager, ProcedureManagerRef, ProcedureState,
ProcedureWithId, Status, StringKey,
PoisonKey, PoisonKeys, Procedure, ProcedureId, ProcedureInfo, ProcedureManager,
ProcedureManagerRef, ProcedureState, ProcedureWithId, Status, StringKey,
};
pub use crate::watcher::Watcher;

View File

@@ -15,7 +15,8 @@
mod runner;
mod rwlock;
use std::collections::{HashMap, VecDeque};
use std::collections::hash_map::Entry;
use std::collections::{HashMap, HashSet, VecDeque};
use std::sync::atomic::{AtomicBool, AtomicI64, Ordering};
use std::sync::{Arc, Mutex, RwLock};
use std::time::{Duration, Instant};
@@ -25,21 +26,23 @@ use backon::ExponentialBuilder;
use common_runtime::{RepeatedTask, TaskFunction};
use common_telemetry::tracing_context::{FutureExt, TracingContext};
use common_telemetry::{error, info, tracing};
use snafu::{ensure, ResultExt};
use snafu::{ensure, OptionExt, ResultExt};
use tokio::sync::watch::{self, Receiver, Sender};
use tokio::sync::{Mutex as TokioMutex, Notify};
use self::rwlock::KeyRwLock;
use crate::error::{
self, DuplicateProcedureSnafu, Error, LoaderConflictSnafu, ManagerNotStartSnafu, Result,
StartRemoveOutdatedMetaTaskSnafu, StopRemoveOutdatedMetaTaskSnafu,
self, DuplicateProcedureSnafu, Error, LoaderConflictSnafu, ManagerNotStartSnafu,
PoisonKeyNotDefinedSnafu, ProcedureNotFoundSnafu, Result, StartRemoveOutdatedMetaTaskSnafu,
StopRemoveOutdatedMetaTaskSnafu, TooManyRunningProceduresSnafu,
};
use crate::local::runner::Runner;
use crate::procedure::{BoxedProcedureLoader, InitProcedureState, ProcedureInfo};
use crate::procedure::{BoxedProcedureLoader, InitProcedureState, PoisonKeys, ProcedureInfo};
use crate::store::poison_store::PoisonStoreRef;
use crate::store::{ProcedureMessage, ProcedureMessages, ProcedureStore, StateStoreRef};
use crate::{
BoxedProcedure, ContextProvider, LockKey, ProcedureId, ProcedureManager, ProcedureState,
ProcedureWithId, Watcher,
BoxedProcedure, ContextProvider, LockKey, PoisonKey, ProcedureId, ProcedureManager,
ProcedureState, ProcedureWithId, Watcher,
};
/// The expired time of a procedure's metadata.
@@ -65,6 +68,8 @@ pub(crate) struct ProcedureMeta {
child_notify: Notify,
/// Lock required by this procedure.
lock_key: LockKey,
/// Poison keys that may cause this procedure to become poisoned during execution.
poison_keys: PoisonKeys,
/// Sender to notify the procedure state.
state_sender: Sender<ProcedureState>,
/// Receiver to watch the procedure state.
@@ -83,6 +88,7 @@ impl ProcedureMeta {
procedure_state: ProcedureState,
parent_id: Option<ProcedureId>,
lock_key: LockKey,
poison_keys: PoisonKeys,
type_name: &str,
) -> ProcedureMeta {
let (state_sender, state_receiver) = watch::channel(procedure_state);
@@ -91,6 +97,7 @@ impl ProcedureMeta {
parent_id,
child_notify: Notify::new(),
lock_key,
poison_keys,
state_sender,
state_receiver,
children: Mutex::new(Vec::new()),
@@ -147,7 +154,6 @@ type ProcedureMetaRef = Arc<ProcedureMeta>;
/// Procedure loaded from store.
struct LoadedProcedure {
procedure: BoxedProcedure,
parent_id: Option<ProcedureId>,
step: u32,
}
@@ -157,12 +163,13 @@ pub(crate) struct ManagerContext {
loaders: Mutex<HashMap<String, BoxedProcedureLoader>>,
key_lock: KeyRwLock<String>,
procedures: RwLock<HashMap<ProcedureId, ProcedureMetaRef>>,
/// Messages loaded from the procedure store.
messages: Mutex<HashMap<ProcedureId, ProcedureMessage>>,
running_procedures: Mutex<HashSet<ProcedureId>>,
/// Ids and finished time of finished procedures.
finished_procedures: Mutex<VecDeque<(ProcedureId, Instant)>>,
/// Running flag.
running: Arc<AtomicBool>,
/// Poison manager.
poison_manager: PoisonStoreRef,
}
#[async_trait]
@@ -170,18 +177,41 @@ impl ContextProvider for ManagerContext {
async fn procedure_state(&self, procedure_id: ProcedureId) -> Result<Option<ProcedureState>> {
Ok(self.state(procedure_id))
}
async fn try_put_poison(&self, key: &PoisonKey, procedure_id: ProcedureId) -> Result<()> {
{
// validate the procedure exists
let procedures = self.procedures.read().unwrap();
let procedure = procedures
.get(&procedure_id)
.context(ProcedureNotFoundSnafu { procedure_id })?;
// validate the poison key is defined
ensure!(
procedure.poison_keys.contains(key),
PoisonKeyNotDefinedSnafu {
key: key.clone(),
procedure_id
}
);
}
let key = key.to_string();
let procedure_id = procedure_id.to_string();
self.poison_manager.try_put_poison(key, procedure_id).await
}
}
impl ManagerContext {
/// Returns a new [ManagerContext].
fn new() -> ManagerContext {
fn new(poison_manager: PoisonStoreRef) -> ManagerContext {
ManagerContext {
key_lock: KeyRwLock::new(),
loaders: Mutex::new(HashMap::new()),
procedures: RwLock::new(HashMap::new()),
messages: Mutex::new(HashMap::new()),
running_procedures: Mutex::new(HashSet::new()),
finished_procedures: Mutex::new(VecDeque::new()),
running: Arc::new(AtomicBool::new(false)),
poison_manager,
}
}
@@ -210,18 +240,27 @@ impl ManagerContext {
procedures.contains_key(&procedure_id)
}
/// Returns the number of running procedures.
fn num_running_procedures(&self) -> usize {
self.running_procedures.lock().unwrap().len()
}
/// Try to insert the `procedure` to the context if there is no procedure
/// with same [ProcedureId].
///
/// Returns `false` if there is already a procedure using the same [ProcedureId].
fn try_insert_procedure(&self, meta: ProcedureMetaRef) -> bool {
let procedure_id = meta.id;
let mut procedures = self.procedures.write().unwrap();
if procedures.contains_key(&meta.id) {
return false;
match procedures.entry(procedure_id) {
Entry::Occupied(_) => return false,
Entry::Vacant(vacant_entry) => {
vacant_entry.insert(meta);
}
}
let old = procedures.insert(meta.id, meta);
debug_assert!(old.is_none());
let mut running_procedures = self.running_procedures.lock().unwrap();
running_procedures.insert(procedure_id);
true
}
@@ -264,16 +303,6 @@ impl ManagerContext {
}
}
/// Load procedure with specific `procedure_id` from cached [ProcedureMessage]s.
fn load_one_procedure(&self, procedure_id: ProcedureId) -> Option<LoadedProcedure> {
let message = {
let messages = self.messages.lock().unwrap();
messages.get(&procedure_id).cloned()?
};
self.load_one_procedure_from_message(procedure_id, &message)
}
/// Load procedure from specific [ProcedureMessage].
fn load_one_procedure_from_message(
&self,
@@ -301,7 +330,6 @@ impl ManagerContext {
Some(LoadedProcedure {
procedure,
parent_id: message.parent_id,
step: message.step,
})
}
@@ -350,23 +378,19 @@ impl ManagerContext {
}
}
/// Remove cached [ProcedureMessage] by ids.
fn remove_messages(&self, procedure_ids: &[ProcedureId]) {
let mut messages = self.messages.lock().unwrap();
for procedure_id in procedure_ids {
let _ = messages.remove(procedure_id);
}
}
/// Clean resources of finished procedures.
fn on_procedures_finish(&self, procedure_ids: &[ProcedureId]) {
self.remove_messages(procedure_ids);
// Since users need to query the procedure state, so we can't remove the
// meta of the procedure directly.
let now = Instant::now();
let mut finished_procedures = self.finished_procedures.lock().unwrap();
finished_procedures.extend(procedure_ids.iter().map(|id| (*id, now)));
// Remove the procedures from the running set.
let mut running_procedures = self.running_procedures.lock().unwrap();
for procedure_id in procedure_ids {
running_procedures.remove(procedure_id);
}
}
/// Remove metadata of outdated procedures.
@@ -410,6 +434,7 @@ pub struct ManagerConfig {
pub retry_delay: Duration,
pub remove_outdated_meta_task_interval: Duration,
pub remove_outdated_meta_ttl: Duration,
pub max_running_procedures: usize,
}
impl Default for ManagerConfig {
@@ -420,6 +445,7 @@ impl Default for ManagerConfig {
retry_delay: Duration::from_millis(500),
remove_outdated_meta_task_interval: Duration::from_secs(60 * 10),
remove_outdated_meta_ttl: META_TTL,
max_running_procedures: 128,
}
}
}
@@ -437,8 +463,12 @@ pub struct LocalManager {
impl LocalManager {
/// Create a new [LocalManager] with specific `config`.
pub fn new(config: ManagerConfig, state_store: StateStoreRef) -> LocalManager {
let manager_ctx = Arc::new(ManagerContext::new());
pub fn new(
config: ManagerConfig,
state_store: StateStoreRef,
poison_store: PoisonStoreRef,
) -> LocalManager {
let manager_ctx = Arc::new(ManagerContext::new(poison_store));
LocalManager {
manager_ctx,
@@ -476,6 +506,7 @@ impl LocalManager {
procedure_state,
None,
procedure.lock_key(),
procedure.poison_keys(),
procedure.type_name(),
));
let runner = Runner {
@@ -492,6 +523,13 @@ impl LocalManager {
let watcher = meta.state_receiver.clone();
ensure!(
self.manager_ctx.num_running_procedures() < self.config.max_running_procedures,
TooManyRunningProceduresSnafu {
max_running_procedures: self.config.max_running_procedures,
}
);
// Inserts meta into the manager before actually spawnd the runner.
ensure!(
self.manager_ctx.try_insert_procedure(meta),
@@ -718,6 +756,7 @@ pub(crate) mod test_util {
ProcedureState::Running,
None,
LockKey::default(),
PoisonKeys::default(),
"ProcedureAdapter",
)
}
@@ -741,11 +780,17 @@ mod tests {
use super::*;
use crate::error::{self, Error};
use crate::store::state_store::ObjectStateStore;
use crate::test_util::InMemoryPoisonStore;
use crate::{Context, Procedure, Status};
fn new_test_manager_context() -> ManagerContext {
let poison_manager = Arc::new(InMemoryPoisonStore::default());
ManagerContext::new(poison_manager)
}
#[test]
fn test_manager_context() {
let ctx = ManagerContext::new();
let ctx = new_test_manager_context();
let meta = Arc::new(test_util::procedure_meta_for_test());
assert!(!ctx.contains_procedure(meta.id));
@@ -761,7 +806,7 @@ mod tests {
#[test]
fn test_manager_context_insert_duplicate() {
let ctx = ManagerContext::new();
let ctx = new_test_manager_context();
let meta = Arc::new(test_util::procedure_meta_for_test());
assert!(ctx.try_insert_procedure(meta.clone()));
@@ -783,7 +828,7 @@ mod tests {
#[test]
fn test_procedures_in_tree() {
let ctx = ManagerContext::new();
let ctx = new_test_manager_context();
let root = Arc::new(test_util::procedure_meta_for_test());
assert!(ctx.try_insert_procedure(root.clone()));
@@ -807,6 +852,7 @@ mod tests {
struct ProcedureToLoad {
content: String,
lock_key: LockKey,
poison_keys: PoisonKeys,
}
#[async_trait]
@@ -826,6 +872,10 @@ mod tests {
fn lock_key(&self) -> LockKey {
self.lock_key.clone()
}
fn poison_keys(&self) -> PoisonKeys {
self.poison_keys.clone()
}
}
impl ProcedureToLoad {
@@ -833,6 +883,7 @@ mod tests {
ProcedureToLoad {
content: content.to_string(),
lock_key: LockKey::default(),
poison_keys: PoisonKeys::default(),
}
}
@@ -855,7 +906,8 @@ mod tests {
..Default::default()
};
let state_store = Arc::new(ObjectStateStore::new(test_util::new_object_store(&dir)));
let manager = LocalManager::new(config, state_store);
let poison_manager = Arc::new(InMemoryPoisonStore::new());
let manager = LocalManager::new(config, state_store, poison_manager);
manager.manager_ctx.start();
manager
@@ -879,7 +931,8 @@ mod tests {
..Default::default()
};
let state_store = Arc::new(ObjectStateStore::new(object_store.clone()));
let manager = LocalManager::new(config, state_store);
let poison_manager = Arc::new(InMemoryPoisonStore::new());
let manager = LocalManager::new(config, state_store, poison_manager);
manager.manager_ctx.start();
manager
@@ -932,7 +985,8 @@ mod tests {
..Default::default()
};
let state_store = Arc::new(ObjectStateStore::new(test_util::new_object_store(&dir)));
let manager = LocalManager::new(config, state_store);
let poison_manager = Arc::new(InMemoryPoisonStore::new());
let manager = LocalManager::new(config, state_store, poison_manager);
manager.manager_ctx.start();
let procedure_id = ProcedureId::random();
@@ -983,7 +1037,8 @@ mod tests {
..Default::default()
};
let state_store = Arc::new(ObjectStateStore::new(test_util::new_object_store(&dir)));
let manager = LocalManager::new(config, state_store);
let poison_manager = Arc::new(InMemoryPoisonStore::new());
let manager = LocalManager::new(config, state_store, poison_manager);
manager.manager_ctx.start();
#[derive(Debug)]
@@ -1022,6 +1077,10 @@ mod tests {
fn lock_key(&self) -> LockKey {
LockKey::single_exclusive("test.submit")
}
fn poison_keys(&self) -> PoisonKeys {
PoisonKeys::default()
}
}
let check_procedure = |procedure| async {
@@ -1059,7 +1118,8 @@ mod tests {
..Default::default()
};
let state_store = Arc::new(ObjectStateStore::new(test_util::new_object_store(&dir)));
let manager = LocalManager::new(config, state_store);
let poison_manager = Arc::new(InMemoryPoisonStore::new());
let manager = LocalManager::new(config, state_store, poison_manager);
let mut procedure = ProcedureToLoad::new("submit");
procedure.lock_key = LockKey::single_exclusive("test.submit");
@@ -1086,7 +1146,8 @@ mod tests {
..Default::default()
};
let state_store = Arc::new(ObjectStateStore::new(test_util::new_object_store(&dir)));
let manager = LocalManager::new(config, state_store);
let poison_manager = Arc::new(InMemoryPoisonStore::new());
let manager = LocalManager::new(config, state_store, poison_manager);
manager.start().await.unwrap();
manager.stop().await.unwrap();
@@ -1119,9 +1180,11 @@ mod tests {
retry_delay: Duration::from_millis(500),
remove_outdated_meta_task_interval: Duration::from_millis(1),
remove_outdated_meta_ttl: Duration::from_millis(1),
max_running_procedures: 128,
};
let state_store = Arc::new(ObjectStateStore::new(object_store.clone()));
let manager = LocalManager::new(config, state_store);
let poison_manager = Arc::new(InMemoryPoisonStore::new());
let manager = LocalManager::new(config, state_store, poison_manager);
manager.manager_ctx.set_running();
let mut procedure = ProcedureToLoad::new("submit");
@@ -1191,11 +1254,76 @@ mod tests {
.is_none());
}
#[tokio::test]
async fn test_too_many_running_procedures() {
let dir = create_temp_dir("too_many_running_procedures");
let config = ManagerConfig {
parent_path: "data/".to_string(),
max_retry_times: 3,
retry_delay: Duration::from_millis(500),
max_running_procedures: 1,
..Default::default()
};
let state_store = Arc::new(ObjectStateStore::new(test_util::new_object_store(&dir)));
let poison_manager = Arc::new(InMemoryPoisonStore::new());
let manager = LocalManager::new(config, state_store, poison_manager);
manager.manager_ctx.set_running();
manager
.manager_ctx
.running_procedures
.lock()
.unwrap()
.insert(ProcedureId::random());
manager.start().await.unwrap();
// Submit a new procedure should fail.
let mut procedure = ProcedureToLoad::new("submit");
procedure.lock_key = LockKey::single_exclusive("test.submit");
let procedure_id = ProcedureId::random();
let err = manager
.submit(ProcedureWithId {
id: procedure_id,
procedure: Box::new(procedure),
})
.await
.unwrap_err();
assert!(matches!(err, Error::TooManyRunningProcedures { .. }));
manager
.manager_ctx
.running_procedures
.lock()
.unwrap()
.clear();
// Submit a new procedure should succeed.
let mut procedure = ProcedureToLoad::new("submit");
procedure.lock_key = LockKey::single_exclusive("test.submit");
assert!(manager
.submit(ProcedureWithId {
id: procedure_id,
procedure: Box::new(procedure),
})
.await
.is_ok());
assert!(manager
.procedure_state(procedure_id)
.await
.unwrap()
.is_some());
// Wait for the procedure done.
let mut watcher = manager.procedure_watcher(procedure_id).unwrap();
watcher.changed().await.unwrap();
assert!(watcher.borrow().is_done());
}
#[derive(Debug)]
struct ProcedureToRecover {
content: String,
lock_key: LockKey,
notify: Option<Arc<Notify>>,
poison_keys: PoisonKeys,
}
#[async_trait]
@@ -1220,6 +1348,10 @@ mod tests {
self.notify.as_ref().unwrap().notify_one();
Ok(())
}
fn poison_keys(&self) -> PoisonKeys {
self.poison_keys.clone()
}
}
impl ProcedureToRecover {
@@ -1227,6 +1359,7 @@ mod tests {
ProcedureToRecover {
content: content.to_string(),
lock_key: LockKey::default(),
poison_keys: PoisonKeys::default(),
notify: None,
}
}
@@ -1236,6 +1369,7 @@ mod tests {
let procedure = ProcedureToRecover {
content: json.to_string(),
lock_key: LockKey::default(),
poison_keys: PoisonKeys::default(),
notify: Some(notify.clone()),
};
Ok(Box::new(procedure) as _)
@@ -1256,7 +1390,8 @@ mod tests {
..Default::default()
};
let state_store = Arc::new(ObjectStateStore::new(object_store.clone()));
let manager = LocalManager::new(config, state_store);
let poison_manager = Arc::new(InMemoryPoisonStore::new());
let manager = LocalManager::new(config, state_store, poison_manager);
manager.manager_ctx.start();
let notify = Arc::new(Notify::new());

View File

@@ -238,11 +238,34 @@ impl Runner {
}
ProcedureState::Done { .. } => return,
ProcedureState::Failed { .. } => return,
ProcedureState::Poisoned { .. } => return,
}
self.execute_once(ctx).await;
}
}
async fn clean_poisons(&mut self) -> Result<()> {
let mut error = None;
for key in self.meta.poison_keys.iter() {
let key = key.to_string();
if let Err(e) = self
.manager_ctx
.poison_manager
.delete_poison(key, self.meta.id.to_string())
.await
{
error!(e; "Failed to clean poisons for procedure: {}", self.meta.id);
error = Some(e);
}
}
// returns the last error if any.
if let Some(e) = error {
return Err(e);
}
Ok(())
}
async fn rollback(&mut self, ctx: &Context, err: Arc<Error>) {
if self.procedure.rollback_supported() {
if let Err(e) = self.procedure.rollback(ctx).await {
@@ -255,7 +278,7 @@ impl Runner {
}
async fn prepare_rollback(&mut self, err: Arc<Error>) {
if let Err(e) = self.write_procedure_state(err.to_string()).await {
if let Err(e) = self.write_rollback_procedure_state(err.to_string()).await {
self.meta
.set_state(ProcedureState::prepare_rollback(Arc::new(e)));
return;
@@ -288,26 +311,48 @@ impl Runner {
return;
}
// Cleans poisons before persist.
if status.need_clean_poisons() {
if let Err(e) = self.clean_poisons().await {
error!(e; "Failed to clean poison for procedure: {}", self.meta.id);
self.meta.set_state(ProcedureState::retrying(Arc::new(e)));
return;
}
}
if status.need_persist() {
if let Err(err) = self.persist_procedure().await {
self.meta.set_state(ProcedureState::retrying(Arc::new(err)));
if let Err(e) = self.persist_procedure().await {
error!(e; "Failed to persist procedure: {}", self.meta.id);
self.meta.set_state(ProcedureState::retrying(Arc::new(e)));
return;
}
}
match status {
Status::Executing { .. } => (),
Status::Executing { .. } => {}
Status::Suspended { subprocedures, .. } => {
self.on_suspended(subprocedures).await;
}
Status::Done { output } => {
if let Err(e) = self.commit_procedure().await {
error!(e; "Failed to commit procedure: {}", self.meta.id);
self.meta.set_state(ProcedureState::retrying(Arc::new(e)));
return;
}
self.done(output);
}
Status::Poisoned { error, keys } => {
error!(
error;
"Procedure {}-{} is poisoned, keys: {:?}",
self.procedure.type_name(),
self.meta.id,
keys,
);
self.meta
.set_state(ProcedureState::poisoned(keys, Arc::new(error)));
}
}
}
Err(e) => {
@@ -327,6 +372,14 @@ impl Runner {
return;
}
if e.need_clean_poisons() {
if let Err(e) = self.clean_poisons().await {
error!(e; "Failed to clean poison for procedure: {}", self.meta.id);
self.meta.set_state(ProcedureState::retrying(Arc::new(e)));
return;
}
}
if e.is_retry_later() {
self.meta.set_state(ProcedureState::retrying(Arc::new(e)));
return;
@@ -339,7 +392,9 @@ impl Runner {
}
ProcedureState::PrepareRollback { error } => self.prepare_rollback(error).await,
ProcedureState::RollingBack { error } => self.rollback(ctx, error).await,
ProcedureState::Failed { .. } | ProcedureState::Done { .. } => (),
ProcedureState::Failed { .. }
| ProcedureState::Done { .. }
| ProcedureState::Poisoned { .. } => (),
}
}
@@ -348,30 +403,21 @@ impl Runner {
&self,
procedure_id: ProcedureId,
procedure_state: ProcedureState,
mut procedure: BoxedProcedure,
procedure: BoxedProcedure,
) {
if self.manager_ctx.contains_procedure(procedure_id) {
// If the parent has already submitted this procedure, don't submit it again.
return;
}
let mut step = 0;
if let Some(loaded_procedure) = self.manager_ctx.load_one_procedure(procedure_id) {
// Try to load procedure state from the message to avoid re-run the subprocedure
// from initial state.
assert_eq!(self.meta.id, loaded_procedure.parent_id.unwrap());
// Use the dumped procedure from the procedure store.
procedure = loaded_procedure.procedure;
// Update step number.
step = loaded_procedure.step;
}
let step = 0;
let meta = Arc::new(ProcedureMeta::new(
procedure_id,
procedure_state,
Some(self.meta.id),
procedure.lock_key(),
procedure.poison_keys(),
procedure.type_name(),
));
let runner = Runner {
@@ -494,7 +540,7 @@ impl Runner {
Ok(())
}
async fn write_procedure_state(&mut self, error: String) -> Result<()> {
async fn write_rollback_procedure_state(&mut self, error: String) -> Result<()> {
// Persists procedure state
let type_name = self.procedure.type_name().to_string();
let data = self.procedure.dump()?;
@@ -549,8 +595,10 @@ mod tests {
use super::*;
use crate::local::test_util;
use crate::procedure::PoisonKeys;
use crate::store::proc_path;
use crate::{ContextProvider, Error, LockKey, Procedure};
use crate::test_util::InMemoryPoisonStore;
use crate::{ContextProvider, Error, LockKey, PoisonKey, Procedure};
const ROOT_ID: &str = "9f805a1f-05f7-490c-9f91-bd56e3cc54c1";
@@ -562,7 +610,9 @@ mod tests {
Runner {
meta,
procedure,
manager_ctx: Arc::new(ManagerContext::new()),
manager_ctx: Arc::new(ManagerContext::new(
Arc::new(InMemoryPoisonStore::default()),
)),
step: 0,
exponential_builder: ExponentialBuilder::default(),
store,
@@ -587,6 +637,16 @@ mod tests {
assert_eq!(files, files_in_dir);
}
fn context_with_provider(
procedure_id: ProcedureId,
provider: Arc<dyn ContextProvider>,
) -> Context {
Context {
procedure_id,
provider,
}
}
fn context_without_provider(procedure_id: ProcedureId) -> Context {
struct MockProvider;
@@ -598,6 +658,14 @@ mod tests {
) -> Result<Option<ProcedureState>> {
unimplemented!()
}
async fn try_put_poison(
&self,
_key: &PoisonKey,
_procedure_id: ProcedureId,
) -> Result<()> {
unimplemented!()
}
}
Context {
@@ -611,6 +679,7 @@ mod tests {
struct ProcedureAdapter<F> {
data: String,
lock_key: LockKey,
poison_keys: PoisonKeys,
exec_fn: F,
rollback_fn: Option<RollbackFn>,
}
@@ -620,6 +689,7 @@ mod tests {
let mut meta = test_util::procedure_meta_for_test();
meta.id = ProcedureId::parse_str(uuid).unwrap();
meta.lock_key = self.lock_key.clone();
meta.poison_keys = self.poison_keys.clone();
Arc::new(meta)
}
@@ -657,6 +727,10 @@ mod tests {
fn lock_key(&self) -> LockKey {
self.lock_key.clone()
}
fn poison_keys(&self) -> PoisonKeys {
self.poison_keys.clone()
}
}
async fn execute_once_normal(persist: bool, first_files: &[&str], second_files: &[&str]) {
@@ -665,7 +739,7 @@ mod tests {
times += 1;
async move {
if times == 1 {
Ok(Status::Executing { persist })
Ok(Status::executing(persist))
} else {
Ok(Status::done())
}
@@ -675,6 +749,7 @@ mod tests {
let normal = ProcedureAdapter {
data: "normal".to_string(),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
poison_keys: PoisonKeys::default(),
exec_fn,
rollback_fn: None,
};
@@ -739,6 +814,7 @@ mod tests {
let suspend = ProcedureAdapter {
data: "suspend".to_string(),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
poison_keys: PoisonKeys::default(),
exec_fn,
rollback_fn: None,
};
@@ -763,7 +839,7 @@ mod tests {
async move {
if times == 1 {
time::sleep(Duration::from_millis(200)).await;
Ok(Status::Executing { persist: true })
Ok(Status::executing(true))
} else {
Ok(Status::done())
}
@@ -773,6 +849,7 @@ mod tests {
let child = ProcedureAdapter {
data: "child".to_string(),
lock_key: LockKey::new_exclusive(keys.iter().map(|k| k.to_string())),
poison_keys: PoisonKeys::default(),
exec_fn,
rollback_fn: None,
};
@@ -842,6 +919,7 @@ mod tests {
let parent = ProcedureAdapter {
data: "parent".to_string(),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
poison_keys: PoisonKeys::default(),
exec_fn,
rollback_fn: None,
};
@@ -853,7 +931,8 @@ mod tests {
let object_store = test_util::new_object_store(&dir);
let procedure_store = Arc::new(ProcedureStore::from_object_store(object_store.clone()));
let mut runner = new_runner(meta.clone(), Box::new(parent), procedure_store.clone());
let manager_ctx = Arc::new(ManagerContext::new());
let poison_manager = Arc::new(InMemoryPoisonStore::default());
let manager_ctx = Arc::new(ManagerContext::new(poison_manager));
manager_ctx.start();
// Manually add this procedure to the manager ctx.
assert!(manager_ctx.try_insert_procedure(meta));
@@ -885,10 +964,11 @@ mod tests {
#[tokio::test]
async fn test_running_is_stopped() {
let exec_fn = move |_| async move { Ok(Status::Executing { persist: true }) }.boxed();
let exec_fn = move |_| async move { Ok(Status::executing(true)) }.boxed();
let normal = ProcedureAdapter {
data: "normal".to_string(),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
poison_keys: PoisonKeys::default(),
exec_fn,
rollback_fn: None,
};
@@ -933,6 +1013,7 @@ mod tests {
let normal = ProcedureAdapter {
data: "fail".to_string(),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
poison_keys: PoisonKeys::default(),
exec_fn,
rollback_fn: None,
};
@@ -959,6 +1040,7 @@ mod tests {
let fail = ProcedureAdapter {
data: "fail".to_string(),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
poison_keys: PoisonKeys::default(),
exec_fn,
rollback_fn: None,
};
@@ -995,6 +1077,7 @@ mod tests {
let fail = ProcedureAdapter {
data: "fail".to_string(),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
poison_keys: PoisonKeys::default(),
exec_fn,
rollback_fn: Some(Box::new(rollback_fn)),
};
@@ -1046,6 +1129,7 @@ mod tests {
let retry_later = ProcedureAdapter {
data: "retry_later".to_string(),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
poison_keys: PoisonKeys::default(),
exec_fn,
rollback_fn: None,
};
@@ -1082,6 +1166,7 @@ mod tests {
let exceed_max_retry_later = ProcedureAdapter {
data: "exceed_max_retry_later".to_string(),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
poison_keys: PoisonKeys::default(),
exec_fn,
rollback_fn: None,
};
@@ -1117,6 +1202,7 @@ mod tests {
let exceed_max_retry_later = ProcedureAdapter {
data: "exceed_max_rollback".to_string(),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
poison_keys: PoisonKeys::default(),
exec_fn,
rollback_fn: Some(Box::new(rollback_fn)),
};
@@ -1159,6 +1245,7 @@ mod tests {
let retry_later = ProcedureAdapter {
data: "rollback_after_retry_fail".to_string(),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
poison_keys: PoisonKeys::default(),
exec_fn,
rollback_fn: Some(Box::new(rollback_fn)),
};
@@ -1203,6 +1290,7 @@ mod tests {
let fail = ProcedureAdapter {
data: "fail".to_string(),
lock_key: LockKey::single_exclusive("catalog.schema.table.region-0"),
poison_keys: PoisonKeys::default(),
exec_fn,
rollback_fn: None,
};
@@ -1238,6 +1326,7 @@ mod tests {
let parent = ProcedureAdapter {
data: "parent".to_string(),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
poison_keys: PoisonKeys::default(),
exec_fn,
rollback_fn: None,
};
@@ -1248,7 +1337,8 @@ mod tests {
let object_store = test_util::new_object_store(&dir);
let procedure_store = Arc::new(ProcedureStore::from_object_store(object_store.clone()));
let mut runner = new_runner(meta.clone(), Box::new(parent), procedure_store);
let manager_ctx = Arc::new(ManagerContext::new());
let poison_manager = Arc::new(InMemoryPoisonStore::default());
let manager_ctx = Arc::new(ManagerContext::new(poison_manager));
manager_ctx.start();
// Manually add this procedure to the manager ctx.
assert!(manager_ctx.try_insert_procedure(meta.clone()));
@@ -1261,4 +1351,327 @@ mod tests {
let err = meta.state().error().unwrap().output_msg();
assert!(err.contains("subprocedure failed"), "{err}");
}
#[tokio::test]
async fn test_execute_with_clean_poisons() {
common_telemetry::init_default_ut_logging();
let mut times = 0;
let poison_key = PoisonKey::new("table/1024");
let moved_poison_key = poison_key.clone();
let exec_fn = move |ctx: Context| {
times += 1;
let poison_key = moved_poison_key.clone();
async move {
if times == 1 {
// Put the poison to the context.
ctx.provider
.try_put_poison(&poison_key, ctx.procedure_id)
.await
.unwrap();
Ok(Status::executing(true))
} else {
Ok(Status::executing_with_clean_poisons(true))
}
}
.boxed()
};
let poison = ProcedureAdapter {
data: "poison".to_string(),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
poison_keys: PoisonKeys::new(vec![poison_key.clone()]),
exec_fn,
rollback_fn: None,
};
let dir = create_temp_dir("clean_poisons");
let meta = poison.new_meta(ROOT_ID);
let object_store = test_util::new_object_store(&dir);
let procedure_store = Arc::new(ProcedureStore::from_object_store(object_store.clone()));
let mut runner = new_runner(meta.clone(), Box::new(poison), procedure_store.clone());
// Use the manager ctx as the context provider.
let ctx = context_with_provider(
meta.id,
runner.manager_ctx.clone() as Arc<dyn ContextProvider>,
);
// Manually add this procedure to the manager ctx.
runner
.manager_ctx
.procedures
.write()
.unwrap()
.insert(meta.id, runner.meta.clone());
runner.manager_ctx.start();
runner.execute_once(&ctx).await;
let state = runner.meta.state();
assert!(state.is_running(), "{state:?}");
let procedure_id = runner
.manager_ctx
.poison_manager
.get_poison(&poison_key.to_string())
.await
.unwrap();
// poison key should be exist.
assert!(procedure_id.is_some());
runner.execute_once(&ctx).await;
let state = runner.meta.state();
assert!(state.is_running(), "{state:?}");
let procedure_id = runner
.manager_ctx
.poison_manager
.get_poison(&poison_key.to_string())
.await
.unwrap();
// poison key should be deleted.
assert!(procedure_id.is_none());
}
#[tokio::test]
async fn test_execute_error_with_clean_poisons() {
common_telemetry::init_default_ut_logging();
let mut times = 0;
let poison_key = PoisonKey::new("table/1024");
let moved_poison_key = poison_key.clone();
let exec_fn = move |ctx: Context| {
times += 1;
let poison_key = moved_poison_key.clone();
async move {
if times == 1 {
// Put the poison to the context.
ctx.provider
.try_put_poison(&poison_key, ctx.procedure_id)
.await
.unwrap();
Ok(Status::executing(true))
} else {
Err(Error::external_and_clean_poisons(MockError::new(
StatusCode::Unexpected,
)))
}
}
.boxed()
};
let poison = ProcedureAdapter {
data: "poison".to_string(),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
poison_keys: PoisonKeys::new(vec![poison_key.clone()]),
exec_fn,
rollback_fn: None,
};
let dir = create_temp_dir("error_with_clean_poisons");
let meta = poison.new_meta(ROOT_ID);
let object_store = test_util::new_object_store(&dir);
let procedure_store = Arc::new(ProcedureStore::from_object_store(object_store.clone()));
let mut runner = new_runner(meta.clone(), Box::new(poison), procedure_store.clone());
// Use the manager ctx as the context provider.
let ctx = context_with_provider(
meta.id,
runner.manager_ctx.clone() as Arc<dyn ContextProvider>,
);
// Manually add this procedure to the manager ctx.
runner
.manager_ctx
.procedures
.write()
.unwrap()
.insert(meta.id, runner.meta.clone());
runner.manager_ctx.start();
runner.execute_once(&ctx).await;
let state = runner.meta.state();
assert!(state.is_running(), "{state:?}");
let procedure_id = runner
.manager_ctx
.poison_manager
.get_poison(&poison_key.to_string())
.await
.unwrap();
// poison key should be exist.
assert!(procedure_id.is_some());
runner.execute_once(&ctx).await;
let state = runner.meta.state();
assert!(state.is_prepare_rollback(), "{state:?}");
let procedure_id = runner
.manager_ctx
.poison_manager
.get_poison(&poison_key.to_string())
.await
.unwrap();
// poison key should be deleted.
assert!(procedure_id.is_none());
}
#[tokio::test]
async fn test_execute_failed_after_set_poison() {
let mut times = 0;
let poison_key = PoisonKey::new("table/1024");
let moved_poison_key = poison_key.clone();
let exec_fn = move |ctx: Context| {
times += 1;
let poison_key = moved_poison_key.clone();
async move {
if times == 1 {
Ok(Status::executing(true))
} else {
// Put the poison to the context.
ctx.provider
.try_put_poison(&poison_key, ctx.procedure_id)
.await
.unwrap();
Err(Error::external(MockError::new(StatusCode::Unexpected)))
}
}
.boxed()
};
let poison = ProcedureAdapter {
data: "poison".to_string(),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
poison_keys: PoisonKeys::new(vec![poison_key.clone()]),
exec_fn,
rollback_fn: None,
};
let dir = create_temp_dir("poison");
let meta = poison.new_meta(ROOT_ID);
let object_store = test_util::new_object_store(&dir);
let procedure_store = Arc::new(ProcedureStore::from_object_store(object_store.clone()));
let mut runner = new_runner(meta.clone(), Box::new(poison), procedure_store.clone());
// Use the manager ctx as the context provider.
let ctx = context_with_provider(
meta.id,
runner.manager_ctx.clone() as Arc<dyn ContextProvider>,
);
// Manually add this procedure to the manager ctx.
runner
.manager_ctx
.procedures
.write()
.unwrap()
.insert(meta.id, runner.meta.clone());
runner.manager_ctx.start();
runner.execute_once(&ctx).await;
let state = runner.meta.state();
assert!(state.is_running(), "{state:?}");
runner.execute_once(&ctx).await;
let state = runner.meta.state();
assert!(state.is_prepare_rollback(), "{state:?}");
assert!(meta.state().is_prepare_rollback());
runner.execute_once(&ctx).await;
let state = runner.meta.state();
assert!(state.is_failed(), "{state:?}");
assert!(meta.state().is_failed());
// Check the poison is set.
let procedure_id = runner
.manager_ctx
.poison_manager
.get_poison(&poison_key.to_string())
.await
.unwrap()
.unwrap();
// If the procedure is poisoned, the poison key shouldn't be deleted.
assert_eq!(&procedure_id.to_string(), ROOT_ID);
}
#[tokio::test]
async fn test_execute_poisoned() {
let mut times = 0;
let poison_key = PoisonKey::new("table/1024");
let moved_poison_key = poison_key.clone();
let exec_fn = move |ctx: Context| {
times += 1;
let poison_key = moved_poison_key.clone();
async move {
if times == 1 {
Ok(Status::executing(true))
} else {
// Put the poison to the context.
ctx.provider
.try_put_poison(&poison_key, ctx.procedure_id)
.await
.unwrap();
Ok(Status::Poisoned {
keys: PoisonKeys::new(vec![poison_key.clone()]),
error: Error::external(MockError::new(StatusCode::Unexpected)),
})
}
}
.boxed()
};
let poison = ProcedureAdapter {
data: "poison".to_string(),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
poison_keys: PoisonKeys::new(vec![poison_key.clone()]),
exec_fn,
rollback_fn: None,
};
let dir = create_temp_dir("poison");
let meta = poison.new_meta(ROOT_ID);
let object_store = test_util::new_object_store(&dir);
let procedure_store = Arc::new(ProcedureStore::from_object_store(object_store.clone()));
let mut runner = new_runner(meta.clone(), Box::new(poison), procedure_store.clone());
// Use the manager ctx as the context provider.
let ctx = context_with_provider(
meta.id,
runner.manager_ctx.clone() as Arc<dyn ContextProvider>,
);
// Manually add this procedure to the manager ctx.
runner
.manager_ctx
.procedures
.write()
.unwrap()
.insert(meta.id, runner.meta.clone());
runner.manager_ctx.start();
runner.execute_once(&ctx).await;
let state = runner.meta.state();
assert!(state.is_running(), "{state:?}");
runner.execute_once(&ctx).await;
let state = runner.meta.state();
assert!(state.is_poisoned(), "{state:?}");
assert!(meta.state().is_poisoned());
check_files(
&object_store,
&procedure_store,
ctx.procedure_id,
&["0000000000.step"],
)
.await;
// Check the poison is set.
let procedure_id = runner
.manager_ctx
.poison_manager
.get_poison(&poison_key.to_string())
.await
.unwrap()
.unwrap();
// If the procedure is poisoned, the poison key shouldn't be deleted.
assert_eq!(procedure_id, ROOT_ID);
}
}

View File

@@ -29,6 +29,8 @@ pub struct ProcedureConfig {
pub retry_delay: Duration,
/// `None` stands for no limit.
pub max_metadata_value_size: Option<ReadableSize>,
/// Max running procedures.
pub max_running_procedures: usize,
}
impl Default for ProcedureConfig {
@@ -37,6 +39,7 @@ impl Default for ProcedureConfig {
max_retry_times: 3,
retry_delay: Duration::from_millis(500),
max_metadata_value_size: None,
max_running_procedures: 128,
}
}
}

View File

@@ -14,6 +14,7 @@
use std::any::Any;
use std::fmt;
use std::fmt::Display;
use std::str::FromStr;
use std::sync::Arc;
@@ -35,6 +36,8 @@ pub enum Status {
Executing {
/// Whether the framework needs to persist the procedure.
persist: bool,
/// Whether the framework needs to clean the poisons.
clean_poisons: bool,
},
/// The procedure has suspended itself and is waiting for subprocedures.
Suspended {
@@ -42,14 +45,40 @@ pub enum Status {
/// Whether the framework needs to persist the procedure.
persist: bool,
},
/// The procedure is poisoned.
Poisoned {
/// The keys that cause the procedure to be poisoned.
keys: PoisonKeys,
/// The error that cause the procedure to be poisoned.
error: Error,
},
/// the procedure is done.
Done { output: Option<Output> },
}
impl Status {
/// Returns a [Status::Poisoned] with given `keys` and `error`.
pub fn poisoned(keys: impl IntoIterator<Item = PoisonKey>, error: Error) -> Status {
Status::Poisoned {
keys: PoisonKeys::new(keys),
error,
}
}
/// Returns a [Status::Executing] with given `persist` flag.
pub fn executing(persist: bool) -> Status {
Status::Executing { persist }
Status::Executing {
persist,
clean_poisons: false,
}
}
/// Returns a [Status::Executing] with given `persist` flag and clean poisons.
pub fn executing_with_clean_poisons(persist: bool) -> Status {
Status::Executing {
persist,
clean_poisons: true,
}
}
/// Returns a [Status::Done] without output.
@@ -86,11 +115,20 @@ impl Status {
/// Returns `true` if the procedure needs the framework to persist its intermediate state.
pub fn need_persist(&self) -> bool {
// If the procedure is done, the framework doesn't need to persist the procedure
// anymore. It only needs to mark the procedure as committed.
match self {
Status::Executing { persist } | Status::Suspended { persist, .. } => *persist,
Status::Done { .. } => false,
// If the procedure is done/poisoned, the framework doesn't need to persist the procedure
// anymore. It only needs to mark the procedure as committed.
Status::Executing { persist, .. } | Status::Suspended { persist, .. } => *persist,
Status::Done { .. } | Status::Poisoned { .. } => false,
}
}
/// Returns `true` if the framework needs to clean the poisons.
pub fn need_clean_poisons(&self) -> bool {
match self {
Status::Executing { clean_poisons, .. } => *clean_poisons,
Status::Done { .. } => true,
_ => false,
}
}
}
@@ -100,6 +138,12 @@ impl Status {
pub trait ContextProvider: Send + Sync {
/// Query the procedure state.
async fn procedure_state(&self, procedure_id: ProcedureId) -> Result<Option<ProcedureState>>;
/// Try to put a poison key for a procedure.
///
/// This method is used to mark a resource as being operated on by a procedure.
/// If the poison key already exists with a different value, the operation will fail.
async fn try_put_poison(&self, key: &PoisonKey, procedure_id: ProcedureId) -> Result<()>;
}
/// Reference-counted pointer to [ContextProvider].
@@ -147,6 +191,11 @@ pub trait Procedure: Send {
/// Returns the [LockKey] that this procedure needs to acquire.
fn lock_key(&self) -> LockKey;
/// Returns the [PoisonKeys] that may cause this procedure to become poisoned during execution.
fn poison_keys(&self) -> PoisonKeys {
PoisonKeys::default()
}
}
#[async_trait]
@@ -174,6 +223,54 @@ impl<T: Procedure + ?Sized> Procedure for Box<T> {
fn lock_key(&self) -> LockKey {
(**self).lock_key()
}
fn poison_keys(&self) -> PoisonKeys {
(**self).poison_keys()
}
}
#[derive(Clone, Debug, PartialEq, Eq, Hash, PartialOrd, Ord)]
pub struct PoisonKey(String);
impl Display for PoisonKey {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
write!(f, "{}", self.0)
}
}
impl PoisonKey {
/// Creates a new [PoisonKey] from a [String].
pub fn new(key: impl Into<String>) -> Self {
Self(key.into())
}
}
/// A collection of [PoisonKey]s.
///
/// This type is used to represent the keys that may cause the procedure to become poisoned during execution.
#[derive(Clone, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, Default)]
pub struct PoisonKeys(SmallVec<[PoisonKey; 2]>);
impl PoisonKeys {
/// Creates a new [PoisonKeys] from a [String].
pub fn single(key: impl Into<String>) -> Self {
Self(smallvec![PoisonKey::new(key)])
}
/// Creates a new [PoisonKeys] from a [PoisonKey].
pub fn new(keys: impl IntoIterator<Item = PoisonKey>) -> Self {
Self(keys.into_iter().collect())
}
/// Returns `true` if the [PoisonKeys] contains the given [PoisonKey].
pub fn contains(&self, key: &PoisonKey) -> bool {
self.0.contains(key)
}
/// Returns an iterator over the [PoisonKey]s.
pub fn iter(&self) -> impl Iterator<Item = &PoisonKey> {
self.0.iter()
}
}
#[derive(Clone, Debug, PartialEq, Eq, Hash, PartialOrd, Ord)]
@@ -325,6 +422,8 @@ pub enum ProcedureState {
RollingBack { error: Arc<Error> },
/// The procedure is failed and cannot proceed anymore.
Failed { error: Arc<Error> },
/// The procedure is poisoned.
Poisoned { keys: PoisonKeys, error: Arc<Error> },
}
impl ProcedureState {
@@ -348,6 +447,11 @@ impl ProcedureState {
ProcedureState::Retrying { error }
}
/// Returns a [ProcedureState] with poisoned state.
pub fn poisoned(keys: PoisonKeys, error: Arc<Error>) -> ProcedureState {
ProcedureState::Poisoned { keys, error }
}
/// Returns true if the procedure state is running.
pub fn is_running(&self) -> bool {
matches!(self, ProcedureState::Running)
@@ -358,6 +462,11 @@ impl ProcedureState {
matches!(self, ProcedureState::Done { .. })
}
/// Returns true if the procedure state is poisoned.
pub fn is_poisoned(&self) -> bool {
matches!(self, ProcedureState::Poisoned { .. })
}
/// Returns true if the procedure state failed.
pub fn is_failed(&self) -> bool {
matches!(self, ProcedureState::Failed { .. })
@@ -384,6 +493,7 @@ impl ProcedureState {
ProcedureState::Failed { error } => Some(error),
ProcedureState::Retrying { error } => Some(error),
ProcedureState::RollingBack { error } => Some(error),
ProcedureState::Poisoned { error, .. } => Some(error),
_ => None,
}
}
@@ -397,6 +507,7 @@ impl ProcedureState {
ProcedureState::Failed { .. } => "Failed",
ProcedureState::PrepareRollback { .. } => "PrepareRollback",
ProcedureState::RollingBack { .. } => "RollingBack",
ProcedureState::Poisoned { .. } => "Poisoned",
}
}
}
@@ -470,12 +581,18 @@ mod tests {
#[test]
fn test_status() {
let status = Status::Executing { persist: false };
let status = Status::executing(false);
assert!(!status.need_persist());
let status = Status::Executing { persist: true };
let status = Status::executing(true);
assert!(status.need_persist());
let status = Status::executing_with_clean_poisons(false);
assert!(status.need_clean_poisons());
let status = Status::executing_with_clean_poisons(true);
assert!(status.need_clean_poisons());
let status = Status::Suspended {
subprocedures: Vec::new(),
persist: false,
@@ -490,6 +607,7 @@ mod tests {
let status = Status::done();
assert!(!status.need_persist());
assert!(status.need_clean_poisons());
}
#[test]

View File

@@ -24,6 +24,7 @@ use crate::error::{Result, ToJsonSnafu};
pub(crate) use crate::store::state_store::StateStoreRef;
use crate::ProcedureId;
pub mod poison_store;
pub mod state_store;
pub mod util;
@@ -341,6 +342,7 @@ mod tests {
use object_store::ObjectStore;
use crate::procedure::PoisonKeys;
use crate::store::state_store::ObjectStateStore;
use crate::BoxedProcedure;
@@ -503,6 +505,10 @@ mod tests {
fn lock_key(&self) -> LockKey {
LockKey::default()
}
fn poison_keys(&self) -> PoisonKeys {
PoisonKeys::default()
}
}
#[tokio::test]

View File

@@ -0,0 +1,59 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use std::sync::Arc;
use async_trait::async_trait;
use crate::error::Result;
pub type PoisonStoreRef = Arc<dyn PoisonStore>;
/// Poison store.
///
/// This trait is used to manage the state of operations on resources, particularly
/// when an operation encounters an unrecoverable error, potentially leading to
/// metadata inconsistency. In such cases, manual intervention is required to
/// resolve the issue before any further operations can be performed on the resource.
///
/// ## Behavior:
/// - **Insertion**: When an operation begins on a resource, a "poison" key is inserted
/// into the state store to indicate the operation is in progress.
/// - **Deletion**: If the operation completes successfully or
/// other cases can ensure the resource is in a consistent state, the poison key is removed
/// from the state store, indicating the resource is in a consistent state.
/// - **Failure Handling**:
/// - If the operation fails or other cases may lead to metadata inconsistency,
/// the poison key remains in the state store.
/// - The presence of this key indicates that the resource has encountered an
/// unrecoverable error and the metadata may be inconsistent.
/// - New operations on the same resource are rejected until the resource is
/// manually recovered and the poison key is removed.
#[async_trait]
pub trait PoisonStore: Send + Sync {
/// Try to put the poison key.
///
/// If the poison key already exists with a different value, the operation will fail.
async fn try_put_poison(&self, key: String, token: String) -> Result<()>;
/// Delete the poison key.
///
/// If the poison key exists with a different value, the operation will fail.
async fn delete_poison(&self, key: String, token: String) -> Result<()>;
/// Get the poison key.
///
/// If the poison key does not exist, the operation will return `None`.
async fn get_poison(&self, key: &str) -> Result<Option<String>>;
}

View File

@@ -0,0 +1,85 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use std::collections::hash_map::Entry;
use std::collections::HashMap;
use std::sync::{Arc, RwLock};
use snafu::ensure;
use super::*;
use crate::error;
use crate::store::poison_store::PoisonStore;
/// A poison store that uses an in-memory map to store the poison state.
#[derive(Debug, Default)]
pub struct InMemoryPoisonStore {
map: Arc<RwLock<HashMap<String, String>>>,
}
impl InMemoryPoisonStore {
/// Create a new in-memory poison manager.
pub fn new() -> Self {
Self::default()
}
}
#[async_trait::async_trait]
impl PoisonStore for InMemoryPoisonStore {
async fn try_put_poison(&self, key: String, token: String) -> Result<()> {
let mut map = self.map.write().unwrap();
match map.entry(key) {
Entry::Vacant(v) => {
v.insert(token.to_string());
}
Entry::Occupied(o) => {
let value = o.get();
ensure!(
value == &token,
error::UnexpectedSnafu {
err_msg: format!("The poison is already set by other token {}", value)
}
);
}
}
Ok(())
}
async fn delete_poison(&self, key: String, token: String) -> Result<()> {
let mut map = self.map.write().unwrap();
match map.entry(key) {
Entry::Vacant(_) => {
// do nothing
}
Entry::Occupied(o) => {
let value = o.get();
ensure!(
value == &token,
error::UnexpectedSnafu {
err_msg: format!("The poison is not set by the token {}", value)
}
);
o.remove();
}
}
Ok(())
}
async fn get_poison(&self, key: &str) -> Result<Option<String>> {
let map = self.map.read().unwrap();
let key = key.to_string();
Ok(map.get(&key).cloned())
}
}

View File

@@ -43,6 +43,10 @@ pub async fn wait(watcher: &mut Watcher) -> Result<Option<Output>> {
ProcedureState::PrepareRollback { error } => {
debug!("commit rollback, source: {}", error)
}
ProcedureState::Poisoned { error, .. } => {
debug!("poisoned, source: {}", error);
return Err(error.clone()).context(ProcedureExecSnafu);
}
}
}
}
@@ -61,7 +65,9 @@ mod tests {
use super::*;
use crate::error::Error;
use crate::local::{test_util, LocalManager, ManagerConfig};
use crate::procedure::PoisonKeys;
use crate::store::state_store::ObjectStateStore;
use crate::test_util::InMemoryPoisonStore;
use crate::{
Context, LockKey, Procedure, ProcedureId, ProcedureManager, ProcedureWithId, Status,
};
@@ -76,7 +82,8 @@ mod tests {
..Default::default()
};
let state_store = Arc::new(ObjectStateStore::new(test_util::new_object_store(&dir)));
let manager = LocalManager::new(config, state_store);
let poison_manager = Arc::new(InMemoryPoisonStore::default());
let manager = LocalManager::new(config, state_store, poison_manager);
manager.start().await.unwrap();
#[derive(Debug)]
@@ -106,6 +113,10 @@ mod tests {
fn lock_key(&self) -> LockKey {
LockKey::single_exclusive("test.submit")
}
fn poison_keys(&self) -> PoisonKeys {
PoisonKeys::default()
}
}
let procedure_id = ProcedureId::random();

View File

@@ -0,0 +1,11 @@
[package]
name = "common-session"
version.workspace = true
edition.workspace = true
license.workspace = true
[lints]
workspace = true
[dependencies]
strum.workspace = true

View File

@@ -0,0 +1,45 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use strum::{AsRefStr, Display, EnumString};
/// Defines the read preference for frontend route operations,
/// determining whether to read from the region leader or follower.
#[derive(Debug, Clone, Copy, Default, EnumString, Display, AsRefStr, PartialEq, Eq)]
pub enum ReadPreference {
#[default]
// Reads all operations from the region leader. This is the default mode.
#[strum(serialize = "leader", to_string = "LEADER")]
Leader,
}
#[cfg(test)]
mod tests {
use std::str::FromStr;
use crate::ReadPreference;
#[test]
fn test_read_preference() {
assert_eq!(ReadPreference::Leader.to_string(), "LEADER");
let read_preference = ReadPreference::from_str("LEADER").unwrap();
assert_eq!(read_preference, ReadPreference::Leader);
let read_preference = ReadPreference::from_str("leader").unwrap();
assert_eq!(read_preference, ReadPreference::Leader);
ReadPreference::from_str("follower").unwrap_err();
}
}

View File

@@ -51,7 +51,6 @@ impl From<DatanodeWalConfig> for MetasrvWalConfig {
DatanodeWalConfig::RaftEngine(_) => Self::RaftEngine,
DatanodeWalConfig::Kafka(config) => Self::Kafka(MetasrvKafkaConfig {
connection: config.connection,
backoff: config.backoff,
kafka_topic: config.kafka_topic,
auto_create_topics: config.auto_create_topics,
}),
@@ -65,7 +64,6 @@ impl From<MetasrvWalConfig> for DatanodeWalConfig {
MetasrvWalConfig::RaftEngine => Self::RaftEngine(RaftEngineConfig::default()),
MetasrvWalConfig::Kafka(config) => Self::Kafka(DatanodeKafkaConfig {
connection: config.connection,
backoff: config.backoff,
kafka_topic: config.kafka_topic,
..Default::default()
}),
@@ -84,7 +82,6 @@ mod tests {
use tests::kafka::common::KafkaTopicConfig;
use super::*;
use crate::config::kafka::common::BackoffConfig;
use crate::config::{DatanodeKafkaConfig, MetasrvKafkaConfig};
use crate::TopicSelectorType;
@@ -175,12 +172,6 @@ mod tests {
client_key_path: None,
}),
},
backoff: BackoffConfig {
init: Duration::from_millis(500),
max: Duration::from_secs(10),
base: 2,
deadline: Some(Duration::from_secs(60 * 5)),
},
kafka_topic: KafkaTopicConfig {
num_topics: 32,
selector_type: TopicSelectorType::RoundRobin,
@@ -212,12 +203,6 @@ mod tests {
},
max_batch_bytes: ReadableSize::mb(1),
consumer_wait_timeout: Duration::from_millis(100),
backoff: BackoffConfig {
init: Duration::from_millis(500),
max: Duration::from_secs(10),
base: 2,
deadline: Some(Duration::from_secs(60 * 5)),
},
kafka_topic: KafkaTopicConfig {
num_topics: 32,
selector_type: TopicSelectorType::RoundRobin,

View File

@@ -17,44 +17,22 @@ use std::sync::Arc;
use std::time::Duration;
use rskafka::client::{Credentials, SaslConfig};
use rskafka::BackoffConfig;
use rustls::{ClientConfig, RootCertStore};
use serde::{Deserialize, Serialize};
use serde_with::with_prefix;
use snafu::{OptionExt, ResultExt};
/// The default backoff config for kafka client.
pub const DEFAULT_BACKOFF_CONFIG: BackoffConfig = BackoffConfig {
init_backoff: Duration::from_millis(100),
max_backoff: Duration::from_secs(10),
base: 2.0,
deadline: Some(Duration::from_secs(120)),
};
use crate::error::{self, Result};
use crate::{TopicSelectorType, BROKER_ENDPOINT, TOPIC_NAME_PREFIX};
with_prefix!(pub backoff_prefix "backoff_");
/// Backoff configurations for kafka client.
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)]
#[serde(default)]
pub struct BackoffConfig {
/// The initial backoff delay.
#[serde(with = "humantime_serde")]
pub init: Duration,
/// The maximum backoff delay.
#[serde(with = "humantime_serde")]
pub max: Duration,
/// The exponential backoff rate, i.e. next backoff = base * current backoff.
pub base: u32,
/// The deadline of retries. `None` stands for no deadline.
#[serde(with = "humantime_serde")]
pub deadline: Option<Duration>,
}
impl Default for BackoffConfig {
fn default() -> Self {
Self {
init: Duration::from_millis(500),
max: Duration::from_secs(10),
base: 2,
deadline: Some(Duration::from_secs(60 * 5)), // 5 mins
}
}
}
/// The SASL configurations for kafka client.
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)]
pub struct KafkaClientSasl {

View File

@@ -18,7 +18,7 @@ use common_base::readable_size::ReadableSize;
use serde::{Deserialize, Serialize};
use super::common::KafkaConnectionConfig;
use crate::config::kafka::common::{backoff_prefix, BackoffConfig, KafkaTopicConfig};
use crate::config::kafka::common::KafkaTopicConfig;
/// Kafka wal configurations for datanode.
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)]
@@ -34,9 +34,6 @@ pub struct DatanodeKafkaConfig {
/// The consumer wait timeout.
#[serde(with = "humantime_serde")]
pub consumer_wait_timeout: Duration,
/// The backoff config.
#[serde(flatten, with = "backoff_prefix")]
pub backoff: BackoffConfig,
/// The kafka topic config.
#[serde(flatten)]
pub kafka_topic: KafkaTopicConfig,
@@ -57,7 +54,6 @@ impl Default for DatanodeKafkaConfig {
// Warning: Kafka has a default limit of 1MB per message in a topic.
max_batch_bytes: ReadableSize::mb(1),
consumer_wait_timeout: Duration::from_millis(100),
backoff: BackoffConfig::default(),
kafka_topic: KafkaTopicConfig::default(),
auto_create_topics: true,
create_index: true,

View File

@@ -15,7 +15,7 @@
use serde::{Deserialize, Serialize};
use super::common::KafkaConnectionConfig;
use crate::config::kafka::common::{backoff_prefix, BackoffConfig, KafkaTopicConfig};
use crate::config::kafka::common::KafkaTopicConfig;
/// Kafka wal configurations for metasrv.
#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)]
@@ -24,9 +24,6 @@ pub struct MetasrvKafkaConfig {
/// The kafka connection config.
#[serde(flatten)]
pub connection: KafkaConnectionConfig,
/// The backoff config.
#[serde(flatten, with = "backoff_prefix")]
pub backoff: BackoffConfig,
/// The kafka config.
#[serde(flatten)]
pub kafka_topic: KafkaTopicConfig,
@@ -38,7 +35,6 @@ impl Default for MetasrvKafkaConfig {
fn default() -> Self {
Self {
connection: Default::default(),
backoff: Default::default(),
kafka_topic: Default::default(),
auto_create_topics: true,
}

View File

@@ -58,17 +58,24 @@ pub struct RegionAliveKeeper {
/// non-decreasing). The heartbeat requests will carry the duration since this epoch, and the
/// duration acts like an "invariant point" for region's keep alive lease.
epoch: Instant,
countdown_task_handler_ext: Option<CountdownTaskHandlerExtRef>,
}
impl RegionAliveKeeper {
/// Returns an empty [RegionAliveKeeper].
pub fn new(region_server: RegionServer, heartbeat_interval_millis: u64) -> Self {
pub fn new(
region_server: RegionServer,
countdown_task_handler_ext: Option<CountdownTaskHandlerExtRef>,
heartbeat_interval_millis: u64,
) -> Self {
Self {
region_server,
tasks: Arc::new(Mutex::new(HashMap::new())),
heartbeat_interval_millis,
started: Arc::new(AtomicBool::new(false)),
epoch: Instant::now(),
countdown_task_handler_ext,
}
}
@@ -85,6 +92,7 @@ impl RegionAliveKeeper {
let handle = Arc::new(CountdownTaskHandle::new(
self.region_server.clone(),
self.countdown_task_handler_ext.clone(),
region_id,
));
@@ -114,7 +122,9 @@ impl RegionAliveKeeper {
for region in regions {
let (role, region_id) = (region.role().into(), RegionId::from(region.region_id));
if let Some(handle) = self.find_handle(region_id).await {
handle.reset_deadline(role, deadline).await;
handle
.reset_deadline(role, deadline, region.extensions.clone())
.await;
} else {
warn!(
"Trying to renew the lease for region {region_id}, the keeper handler is not found!"
@@ -265,13 +275,28 @@ enum CountdownCommand {
/// 4 * `heartbeat_interval_millis`
Start(u64),
/// Reset countdown deadline to the given instance.
/// (NextRole, Deadline)
Reset((RegionRole, Instant)),
/// (NextRole, Deadline, ExtensionInfo)
Reset((RegionRole, Instant, HashMap<String, Vec<u8>>)),
/// Returns the current deadline of the countdown task.
#[cfg(test)]
Deadline(oneshot::Sender<Instant>),
}
pub type CountdownTaskHandlerExtRef = Arc<dyn CountdownTaskHandlerExt>;
/// Extension trait for [CountdownTaskHandlerExt] to reset deadline of a region.
#[async_trait]
pub trait CountdownTaskHandlerExt: Send + Sync {
async fn reset_deadline(
&self,
region_server: &RegionServer,
region_id: RegionId,
role: RegionRole,
deadline: Instant,
extension_info: HashMap<String, Vec<u8>>,
);
}
struct CountdownTaskHandle {
tx: mpsc::Sender<CountdownCommand>,
handler: JoinHandle<()>,
@@ -280,11 +305,16 @@ struct CountdownTaskHandle {
impl CountdownTaskHandle {
/// Creates a new [CountdownTaskHandle] and starts the countdown task.
fn new(region_server: RegionServer, region_id: RegionId) -> Self {
fn new(
region_server: RegionServer,
handler_ext: Option<CountdownTaskHandlerExtRef>,
region_id: RegionId,
) -> Self {
let (tx, rx) = mpsc::channel(1024);
let mut countdown_task = CountdownTask {
region_server,
handler_ext,
region_id,
rx,
};
@@ -323,10 +353,15 @@ impl CountdownTaskHandle {
None
}
async fn reset_deadline(&self, role: RegionRole, deadline: Instant) {
async fn reset_deadline(
&self,
role: RegionRole,
deadline: Instant,
extension_info: HashMap<String, Vec<u8>>,
) {
if let Err(e) = self
.tx
.send(CountdownCommand::Reset((role, deadline)))
.send(CountdownCommand::Reset((role, deadline, extension_info)))
.await
{
warn!(
@@ -350,6 +385,7 @@ impl Drop for CountdownTaskHandle {
struct CountdownTask {
region_server: RegionServer,
region_id: RegionId,
handler_ext: Option<CountdownTaskHandlerExtRef>,
rx: mpsc::Receiver<CountdownCommand>,
}
@@ -379,8 +415,19 @@ impl CountdownTask {
started = true;
}
},
Some(CountdownCommand::Reset((role, deadline))) => {
let _ = self.region_server.set_region_role(self.region_id, role);
Some(CountdownCommand::Reset((role, deadline, extension_info))) => {
if let Err(err) = self.region_server.set_region_role(self.region_id, role) {
error!(err; "Failed to set region role to {role} for region {region_id}");
}
if let Some(ext_handler) = self.handler_ext.as_ref() {
ext_handler.reset_deadline(
&self.region_server,
self.region_id,
role,
deadline,
extension_info,
).await;
}
trace!(
"Reset deadline of region {region_id} to approximately {} seconds later.",
(deadline - Instant::now()).as_secs_f32(),
@@ -402,7 +449,9 @@ impl CountdownTask {
}
() = &mut countdown => {
warn!("The region {region_id} lease is expired, convert region to follower.");
let _ = self.region_server.set_region_role(self.region_id, RegionRole::Follower);
if let Err(err) = self.region_server.set_region_role(self.region_id, RegionRole::Follower) {
error!(err; "Failed to set region role to follower for region {region_id}");
}
// resets the countdown.
let far_future = Instant::now() + Duration::from_secs(86400 * 365 * 30);
countdown.as_mut().reset(far_future);
@@ -431,7 +480,7 @@ mod test {
let engine = Arc::new(engine);
region_server.register_engine(engine.clone());
let alive_keeper = Arc::new(RegionAliveKeeper::new(region_server.clone(), 100));
let alive_keeper = Arc::new(RegionAliveKeeper::new(region_server.clone(), None, 100));
let region_id = RegionId::new(1024, 1);
let builder = CreateRequestBuilder::new();
@@ -468,6 +517,7 @@ mod test {
&[GrantedRegion {
region_id: region_id.as_u64(),
role: api::v1::meta::RegionRole::Leader.into(),
extensions: HashMap::new(),
}],
Instant::now() + Duration::from_millis(200),
)
@@ -492,7 +542,8 @@ mod test {
async fn countdown_task() {
let region_server = mock_region_server();
let countdown_handle = CountdownTaskHandle::new(region_server, RegionId::new(9999, 2));
let countdown_handle =
CountdownTaskHandle::new(region_server, None, RegionId::new(9999, 2));
// If countdown task is not started, its deadline is set to far future.
assert!(
@@ -522,6 +573,7 @@ mod test {
.reset_deadline(
RegionRole::Leader,
Instant::now() + Duration::from_millis(heartbeat_interval_millis * 5),
HashMap::new(),
)
.await;
assert!(

View File

@@ -31,7 +31,6 @@ use servers::export_metrics::ExportMetricsOption;
use servers::grpc::GrpcOptions;
use servers::heartbeat_options::HeartbeatOptions;
use servers::http::HttpOptions;
use servers::Mode;
pub const DEFAULT_OBJECT_STORE_CACHE_SIZE: ReadableSize = ReadableSize::gb(5);
@@ -359,7 +358,6 @@ impl Default for ObjectStoreConfig {
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)]
#[serde(default)]
pub struct DatanodeOptions {
pub mode: Mode,
pub node_id: Option<u64>,
pub require_lease_before_startup: bool,
pub init_regions_in_background: bool,
@@ -395,7 +393,6 @@ impl Default for DatanodeOptions {
#[allow(deprecated)]
fn default() -> Self {
Self {
mode: Mode::Standalone,
node_id: None,
require_lease_before_startup: false,
init_regions_in_background: false,

View File

@@ -157,6 +157,7 @@ impl Datanode {
pub struct DatanodeBuilder {
opts: DatanodeOptions,
mode: Mode,
plugins: Plugins,
meta_client: Option<MetaClientRef>,
kv_backend: Option<KvBackendRef>,
@@ -166,9 +167,10 @@ pub struct DatanodeBuilder {
impl DatanodeBuilder {
/// `kv_backend` is optional. If absent, the builder will try to build one
/// by using the given `opts`
pub fn new(opts: DatanodeOptions, plugins: Plugins) -> Self {
pub fn new(opts: DatanodeOptions, plugins: Plugins, mode: Mode) -> Self {
Self {
opts,
mode,
plugins,
meta_client: None,
kv_backend: None,
@@ -198,7 +200,7 @@ impl DatanodeBuilder {
}
pub async fn build(mut self) -> Result<Datanode> {
let mode = &self.opts.mode;
let mode = &self.mode;
let node_id = self.opts.node_id.context(MissingNodeIdSnafu)?;
let meta_client = self.meta_client.take();
@@ -263,6 +265,7 @@ impl DatanodeBuilder {
region_server.clone(),
meta_client,
cache_registry,
self.plugins.clone(),
)
.await?,
)
@@ -629,6 +632,7 @@ mod tests {
use common_meta::kv_backend::memory::MemoryKvBackend;
use common_meta::kv_backend::KvBackendRef;
use mito2::engine::MITO_ENGINE_NAME;
use servers::Mode;
use store_api::region_request::RegionRequest;
use store_api::storage::RegionId;
@@ -674,6 +678,7 @@ mod tests {
..Default::default()
},
Plugins::default(),
Mode::Standalone,
)
.with_cache_registry(layered_cache_registry);

View File

@@ -18,6 +18,7 @@ use std::sync::Arc;
use std::time::Duration;
use api::v1::meta::{HeartbeatRequest, NodeInfo, Peer, RegionRole, RegionStat};
use common_base::Plugins;
use common_meta::cache_invalidator::CacheInvalidatorRef;
use common_meta::datanode::REGION_STATISTIC_KEY;
use common_meta::distributed_time_constants::META_KEEP_ALIVE_INTERVAL_SECS;
@@ -37,7 +38,7 @@ use tokio::sync::{mpsc, Notify};
use tokio::time::Instant;
use self::handler::RegionHeartbeatResponseHandler;
use crate::alive_keeper::RegionAliveKeeper;
use crate::alive_keeper::{CountdownTaskHandlerExtRef, RegionAliveKeeper};
use crate::config::DatanodeOptions;
use crate::error::{self, MetaClientInitSnafu, Result};
use crate::event_listener::RegionServerEventReceiver;
@@ -73,9 +74,12 @@ impl HeartbeatTask {
region_server: RegionServer,
meta_client: MetaClientRef,
cache_invalidator: CacheInvalidatorRef,
plugins: Plugins,
) -> Result<Self> {
let countdown_task_handler_ext = plugins.get::<CountdownTaskHandlerExtRef>();
let region_alive_keeper = Arc::new(RegionAliveKeeper::new(
region_server.clone(),
countdown_task_handler_ext,
opts.heartbeat.interval.as_millis() as u64,
));
let resp_handler_executor = Arc::new(HandlerGroupExecutor::new(vec![

View File

@@ -26,6 +26,7 @@ use store_api::storage::RegionId;
mod close_region;
mod downgrade_region;
mod flush_region;
mod open_region;
mod upgrade_region;
@@ -42,7 +43,7 @@ pub struct RegionHeartbeatResponseHandler {
/// Handler of the instruction.
pub type InstructionHandler =
Box<dyn FnOnce(HandlerContext) -> BoxFuture<'static, InstructionReply> + Send>;
Box<dyn FnOnce(HandlerContext) -> BoxFuture<'static, Option<InstructionReply>> + Send>;
#[derive(Clone)]
pub struct HandlerContext {
@@ -94,6 +95,9 @@ impl RegionHeartbeatResponseHandler {
handler_context.handle_upgrade_region_instruction(upgrade_region)
})),
Instruction::InvalidateCaches(_) => InvalidHeartbeatResponseSnafu.fail(),
Instruction::FlushRegion(flush_regions) => Ok(Box::new(move |handler_context| {
handler_context.handle_flush_region_instruction(flush_regions)
})),
}
}
}
@@ -129,8 +133,10 @@ impl HeartbeatResponseHandler for RegionHeartbeatResponseHandler {
})
.await;
if let Err(e) = mailbox.send((meta, reply)).await {
error!(e; "Failed to send reply to mailbox");
if let Some(reply) = reply {
if let Err(e) = mailbox.send((meta, reply)).await {
error!(e; "Failed to send reply to mailbox");
}
}
});

View File

@@ -26,28 +26,28 @@ impl HandlerContext {
pub(crate) fn handle_close_region_instruction(
self,
region_ident: RegionIdent,
) -> BoxFuture<'static, InstructionReply> {
) -> BoxFuture<'static, Option<InstructionReply>> {
Box::pin(async move {
let region_id = Self::region_ident_to_region_id(&region_ident);
let request = RegionRequest::Close(RegionCloseRequest {});
let result = self.region_server.handle_request(region_id, request).await;
match result {
Ok(_) => InstructionReply::CloseRegion(SimpleReply {
Ok(_) => Some(InstructionReply::CloseRegion(SimpleReply {
result: true,
error: None,
}),
})),
Err(error::Error::RegionNotFound { .. }) => {
warn!("Received a close region instruction from meta, but target region:{region_id} is not found.");
InstructionReply::CloseRegion(SimpleReply {
Some(InstructionReply::CloseRegion(SimpleReply {
result: true,
error: None,
})
}))
}
Err(err) => InstructionReply::CloseRegion(SimpleReply {
Err(err) => Some(InstructionReply::CloseRegion(SimpleReply {
result: false,
error: Some(format!("{err:?}")),
}),
})),
}
})
}

View File

@@ -24,31 +24,34 @@ use crate::heartbeat::handler::HandlerContext;
use crate::heartbeat::task_tracker::WaitResult;
impl HandlerContext {
async fn downgrade_to_follower_gracefully(&self, region_id: RegionId) -> InstructionReply {
async fn downgrade_to_follower_gracefully(
&self,
region_id: RegionId,
) -> Option<InstructionReply> {
match self
.region_server
.set_region_role_state_gracefully(region_id, SettableRegionRoleState::Follower)
.await
{
Ok(SetRegionRoleStateResponse::Success { last_entry_id }) => {
InstructionReply::DowngradeRegion(DowngradeRegionReply {
Some(InstructionReply::DowngradeRegion(DowngradeRegionReply {
last_entry_id,
exists: true,
error: None,
})
}))
}
Ok(SetRegionRoleStateResponse::NotFound) => {
InstructionReply::DowngradeRegion(DowngradeRegionReply {
Some(InstructionReply::DowngradeRegion(DowngradeRegionReply {
last_entry_id: None,
exists: false,
error: None,
})
}))
}
Err(err) => InstructionReply::DowngradeRegion(DowngradeRegionReply {
Err(err) => Some(InstructionReply::DowngradeRegion(DowngradeRegionReply {
last_entry_id: None,
exists: true,
error: Some(format!("{err:?}")),
}),
})),
}
}
@@ -59,15 +62,15 @@ impl HandlerContext {
flush_timeout,
reject_write,
}: DowngradeRegion,
) -> BoxFuture<'static, InstructionReply> {
) -> BoxFuture<'static, Option<InstructionReply>> {
Box::pin(async move {
let Some(writable) = self.region_server.is_region_leader(region_id) else {
warn!("Region: {region_id} is not found");
return InstructionReply::DowngradeRegion(DowngradeRegionReply {
return Some(InstructionReply::DowngradeRegion(DowngradeRegionReply {
last_entry_id: None,
exists: false,
error: None,
});
}));
};
let region_server_moved = self.region_server.clone();
@@ -99,19 +102,19 @@ impl HandlerContext {
Ok(SetRegionRoleStateResponse::Success { .. }) => {}
Ok(SetRegionRoleStateResponse::NotFound) => {
warn!("Region: {region_id} is not found");
return InstructionReply::DowngradeRegion(DowngradeRegionReply {
return Some(InstructionReply::DowngradeRegion(DowngradeRegionReply {
last_entry_id: None,
exists: false,
error: None,
});
}));
}
Err(err) => {
warn!(err; "Failed to convert region to downgrading leader");
return InstructionReply::DowngradeRegion(DowngradeRegionReply {
return Some(InstructionReply::DowngradeRegion(DowngradeRegionReply {
last_entry_id: None,
exists: true,
error: Some(format!("{err:?}")),
});
}));
}
}
}
@@ -144,18 +147,20 @@ impl HandlerContext {
let result = self.catchup_tasks.wait(&mut watcher, flush_timeout).await;
match result {
WaitResult::Timeout => InstructionReply::DowngradeRegion(DowngradeRegionReply {
last_entry_id: None,
exists: true,
error: Some(format!("Flush region: {region_id} is timeout")),
}),
WaitResult::Timeout => {
Some(InstructionReply::DowngradeRegion(DowngradeRegionReply {
last_entry_id: None,
exists: true,
error: Some(format!("Flush region: {region_id} is timeout")),
}))
}
WaitResult::Finish(Ok(_)) => self.downgrade_to_follower_gracefully(region_id).await,
WaitResult::Finish(Err(err)) => {
InstructionReply::DowngradeRegion(DowngradeRegionReply {
Some(InstructionReply::DowngradeRegion(DowngradeRegionReply {
last_entry_id: None,
exists: true,
error: Some(format!("{err:?}")),
})
}))
}
}
})
@@ -196,9 +201,9 @@ mod tests {
reject_write: false,
})
.await;
assert_matches!(reply, InstructionReply::DowngradeRegion(_));
assert_matches!(reply, Some(InstructionReply::DowngradeRegion(_)));
if let InstructionReply::DowngradeRegion(reply) = reply {
if let InstructionReply::DowngradeRegion(reply) = reply.unwrap() {
assert!(!reply.exists);
assert!(reply.error.is_none());
assert!(reply.last_entry_id.is_none());
@@ -238,9 +243,9 @@ mod tests {
reject_write: false,
})
.await;
assert_matches!(reply, InstructionReply::DowngradeRegion(_));
assert_matches!(reply, Some(InstructionReply::DowngradeRegion(_)));
if let InstructionReply::DowngradeRegion(reply) = reply {
if let InstructionReply::DowngradeRegion(reply) = reply.unwrap() {
assert!(reply.exists);
assert!(reply.error.is_none());
assert_eq!(reply.last_entry_id.unwrap(), 1024);
@@ -272,9 +277,9 @@ mod tests {
reject_write: false,
})
.await;
assert_matches!(reply, InstructionReply::DowngradeRegion(_));
assert_matches!(reply, Some(InstructionReply::DowngradeRegion(_)));
if let InstructionReply::DowngradeRegion(reply) = reply {
if let InstructionReply::DowngradeRegion(reply) = reply.unwrap() {
assert!(reply.exists);
assert!(reply.error.unwrap().contains("timeout"));
assert!(reply.last_entry_id.is_none());
@@ -310,8 +315,8 @@ mod tests {
reject_write: false,
})
.await;
assert_matches!(reply, InstructionReply::DowngradeRegion(_));
if let InstructionReply::DowngradeRegion(reply) = reply {
assert_matches!(reply, Some(InstructionReply::DowngradeRegion(_)));
if let InstructionReply::DowngradeRegion(reply) = reply.unwrap() {
assert!(reply.exists);
assert!(reply.error.unwrap().contains("timeout"));
assert!(reply.last_entry_id.is_none());
@@ -325,11 +330,11 @@ mod tests {
reject_write: false,
})
.await;
assert_matches!(reply, InstructionReply::DowngradeRegion(_));
assert_matches!(reply, Some(InstructionReply::DowngradeRegion(_)));
// Must less than 300 ms.
assert!(timer.elapsed().as_millis() < 300);
if let InstructionReply::DowngradeRegion(reply) = reply {
if let InstructionReply::DowngradeRegion(reply) = reply.unwrap() {
assert!(reply.exists);
assert!(reply.error.is_none());
assert_eq!(reply.last_entry_id.unwrap(), 1024);
@@ -371,8 +376,8 @@ mod tests {
reject_write: false,
})
.await;
assert_matches!(reply, InstructionReply::DowngradeRegion(_));
if let InstructionReply::DowngradeRegion(reply) = reply {
assert_matches!(reply, Some(InstructionReply::DowngradeRegion(_)));
if let InstructionReply::DowngradeRegion(reply) = reply.unwrap() {
assert!(reply.exists);
assert!(reply.error.unwrap().contains("timeout"));
assert!(reply.last_entry_id.is_none());
@@ -386,11 +391,11 @@ mod tests {
reject_write: false,
})
.await;
assert_matches!(reply, InstructionReply::DowngradeRegion(_));
assert_matches!(reply, Some(InstructionReply::DowngradeRegion(_)));
// Must less than 300 ms.
assert!(timer.elapsed().as_millis() < 300);
if let InstructionReply::DowngradeRegion(reply) = reply {
if let InstructionReply::DowngradeRegion(reply) = reply.unwrap() {
assert!(reply.exists);
assert!(reply.error.unwrap().contains("flush failed"));
assert!(reply.last_entry_id.is_none());
@@ -417,8 +422,8 @@ mod tests {
reject_write: false,
})
.await;
assert_matches!(reply, InstructionReply::DowngradeRegion(_));
if let InstructionReply::DowngradeRegion(reply) = reply {
assert_matches!(reply, Some(InstructionReply::DowngradeRegion(_)));
if let InstructionReply::DowngradeRegion(reply) = reply.unwrap() {
assert!(!reply.exists);
assert!(reply.error.is_none());
assert!(reply.last_entry_id.is_none());
@@ -449,8 +454,8 @@ mod tests {
reject_write: false,
})
.await;
assert_matches!(reply, InstructionReply::DowngradeRegion(_));
if let InstructionReply::DowngradeRegion(reply) = reply {
assert_matches!(reply, Some(InstructionReply::DowngradeRegion(_)));
if let InstructionReply::DowngradeRegion(reply) = reply.unwrap() {
assert!(reply.exists);
assert!(reply
.error

View File

@@ -0,0 +1,104 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use common_meta::instruction::{FlushRegions, InstructionReply};
use common_telemetry::warn;
use futures_util::future::BoxFuture;
use store_api::region_request::{RegionFlushRequest, RegionRequest};
use crate::error;
use crate::heartbeat::handler::HandlerContext;
impl HandlerContext {
pub(crate) fn handle_flush_region_instruction(
self,
flush_regions: FlushRegions,
) -> BoxFuture<'static, Option<InstructionReply>> {
Box::pin(async move {
for region_id in flush_regions.region_ids {
let request = RegionRequest::Flush(RegionFlushRequest {
row_group_size: None,
});
let result = self.region_server.handle_request(region_id, request).await;
match result {
Ok(_) => {}
Err(error::Error::RegionNotFound { .. }) => {
warn!("Received a flush region instruction from meta, but target region: {region_id} is not found.");
}
Err(err) => {
warn!(
"Failed to flush region: {region_id}, error: {err}",
region_id = region_id,
err = err,
);
}
}
}
None
})
}
}
#[cfg(test)]
mod tests {
use std::sync::{Arc, RwLock};
use common_meta::instruction::FlushRegions;
use mito2::engine::MITO_ENGINE_NAME;
use store_api::storage::RegionId;
use super::*;
use crate::tests::{mock_region_server, MockRegionEngine};
#[tokio::test]
async fn test_handle_flush_region_instruction() {
let flushed_region_ids: Arc<RwLock<Vec<RegionId>>> = Arc::new(RwLock::new(Vec::new()));
let mock_region_server = mock_region_server();
let region_ids = (0..16).map(|i| RegionId::new(1024, i)).collect::<Vec<_>>();
for region_id in &region_ids {
let flushed_region_ids_ref = flushed_region_ids.clone();
let (mock_engine, _) =
MockRegionEngine::with_custom_apply_fn(MITO_ENGINE_NAME, move |region_engine| {
region_engine.handle_request_mock_fn =
Some(Box::new(move |region_id, _request| {
flushed_region_ids_ref.write().unwrap().push(region_id);
Ok(0)
}))
});
mock_region_server.register_test_region(*region_id, mock_engine);
}
let handler_context = HandlerContext::new_for_test(mock_region_server);
let reply = handler_context
.clone()
.handle_flush_region_instruction(FlushRegions {
region_ids: region_ids.clone(),
})
.await;
assert!(reply.is_none());
assert_eq!(*flushed_region_ids.read().unwrap(), region_ids);
flushed_region_ids.write().unwrap().clear();
let not_found_region_ids = (0..2).map(|i| RegionId::new(2048, i)).collect::<Vec<_>>();
let reply = handler_context
.handle_flush_region_instruction(FlushRegions {
region_ids: not_found_region_ids.clone(),
})
.await;
assert!(reply.is_none());
assert!(flushed_region_ids.read().unwrap().is_empty());
}
}

View File

@@ -30,7 +30,7 @@ impl HandlerContext {
region_wal_options,
skip_wal_replay,
}: OpenRegion,
) -> BoxFuture<'static, InstructionReply> {
) -> BoxFuture<'static, Option<InstructionReply>> {
Box::pin(async move {
let region_id = Self::region_ident_to_region_id(&region_ident);
prepare_wal_options(&mut region_options, region_id, &region_wal_options);
@@ -43,10 +43,10 @@ impl HandlerContext {
let result = self.region_server.handle_request(region_id, request).await;
let success = result.is_ok();
let error = result.as_ref().map_err(|e| format!("{e:?}")).err();
InstructionReply::OpenRegion(SimpleReply {
Some(InstructionReply::OpenRegion(SimpleReply {
result: success,
error,
})
}))
})
}
}

View File

@@ -29,22 +29,22 @@ impl HandlerContext {
replay_timeout,
location_id,
}: UpgradeRegion,
) -> BoxFuture<'static, InstructionReply> {
) -> BoxFuture<'static, Option<InstructionReply>> {
Box::pin(async move {
let Some(writable) = self.region_server.is_region_leader(region_id) else {
return InstructionReply::UpgradeRegion(UpgradeRegionReply {
return Some(InstructionReply::UpgradeRegion(UpgradeRegionReply {
ready: false,
exists: false,
error: None,
});
}));
};
if writable {
return InstructionReply::UpgradeRegion(UpgradeRegionReply {
return Some(InstructionReply::UpgradeRegion(UpgradeRegionReply {
ready: true,
exists: true,
error: None,
});
}));
}
let region_server_moved = self.region_server.clone();
@@ -79,11 +79,11 @@ impl HandlerContext {
// Returns immediately
let Some(replay_timeout) = replay_timeout else {
return InstructionReply::UpgradeRegion(UpgradeRegionReply {
return Some(InstructionReply::UpgradeRegion(UpgradeRegionReply {
ready: false,
exists: true,
error: None,
});
}));
};
// We don't care that it returns a newly registered or running task.
@@ -91,22 +91,24 @@ impl HandlerContext {
let result = self.catchup_tasks.wait(&mut watcher, replay_timeout).await;
match result {
WaitResult::Timeout => InstructionReply::UpgradeRegion(UpgradeRegionReply {
WaitResult::Timeout => Some(InstructionReply::UpgradeRegion(UpgradeRegionReply {
ready: false,
exists: true,
error: None,
}),
WaitResult::Finish(Ok(_)) => InstructionReply::UpgradeRegion(UpgradeRegionReply {
ready: true,
exists: true,
error: None,
}),
})),
WaitResult::Finish(Ok(_)) => {
Some(InstructionReply::UpgradeRegion(UpgradeRegionReply {
ready: true,
exists: true,
error: None,
}))
}
WaitResult::Finish(Err(err)) => {
InstructionReply::UpgradeRegion(UpgradeRegionReply {
Some(InstructionReply::UpgradeRegion(UpgradeRegionReply {
ready: false,
exists: true,
error: Some(format!("{err:?}")),
})
}))
}
}
})
@@ -149,9 +151,9 @@ mod tests {
location_id: None,
})
.await;
assert_matches!(reply, InstructionReply::UpgradeRegion(_));
assert_matches!(reply, Some(InstructionReply::UpgradeRegion(_)));
if let InstructionReply::UpgradeRegion(reply) = reply {
if let InstructionReply::UpgradeRegion(reply) = reply.unwrap() {
assert!(!reply.exists);
assert!(reply.error.is_none());
}
@@ -187,9 +189,9 @@ mod tests {
location_id: None,
})
.await;
assert_matches!(reply, InstructionReply::UpgradeRegion(_));
assert_matches!(reply, Some(InstructionReply::UpgradeRegion(_)));
if let InstructionReply::UpgradeRegion(reply) = reply {
if let InstructionReply::UpgradeRegion(reply) = reply.unwrap() {
assert!(reply.ready);
assert!(reply.exists);
assert!(reply.error.is_none());
@@ -226,9 +228,9 @@ mod tests {
location_id: None,
})
.await;
assert_matches!(reply, InstructionReply::UpgradeRegion(_));
assert_matches!(reply, Some(InstructionReply::UpgradeRegion(_)));
if let InstructionReply::UpgradeRegion(reply) = reply {
if let InstructionReply::UpgradeRegion(reply) = reply.unwrap() {
assert!(!reply.ready);
assert!(reply.exists);
assert!(reply.error.is_none());
@@ -268,9 +270,9 @@ mod tests {
location_id: None,
})
.await;
assert_matches!(reply, InstructionReply::UpgradeRegion(_));
assert_matches!(reply, Some(InstructionReply::UpgradeRegion(_)));
if let InstructionReply::UpgradeRegion(reply) = reply {
if let InstructionReply::UpgradeRegion(reply) = reply.unwrap() {
assert!(!reply.ready);
assert!(reply.exists);
assert!(reply.error.is_none());
@@ -286,11 +288,11 @@ mod tests {
location_id: None,
})
.await;
assert_matches!(reply, InstructionReply::UpgradeRegion(_));
assert_matches!(reply, Some(InstructionReply::UpgradeRegion(_)));
// Must less than 300 ms.
assert!(timer.elapsed().as_millis() < 300);
if let InstructionReply::UpgradeRegion(reply) = reply {
if let InstructionReply::UpgradeRegion(reply) = reply.unwrap() {
assert!(reply.ready);
assert!(reply.exists);
assert!(reply.error.is_none());
@@ -328,10 +330,10 @@ mod tests {
location_id: None,
})
.await;
assert_matches!(reply, InstructionReply::UpgradeRegion(_));
assert_matches!(reply, Some(InstructionReply::UpgradeRegion(_)));
// It didn't wait for handle returns; it had no idea about the error.
if let InstructionReply::UpgradeRegion(reply) = reply {
if let InstructionReply::UpgradeRegion(reply) = reply.unwrap() {
assert!(!reply.ready);
assert!(reply.exists);
assert!(reply.error.is_none());
@@ -346,9 +348,9 @@ mod tests {
location_id: None,
})
.await;
assert_matches!(reply, InstructionReply::UpgradeRegion(_));
assert_matches!(reply, Some(InstructionReply::UpgradeRegion(_)));
if let InstructionReply::UpgradeRegion(reply) = reply {
if let InstructionReply::UpgradeRegion(reply) = reply.unwrap() {
assert!(!reply.ready);
assert!(reply.exists);
assert!(reply.error.is_some());

View File

@@ -55,7 +55,7 @@ use store_api::metric_engine_consts::{
FILE_ENGINE_NAME, LOGICAL_TABLE_METADATA_KEY, METRIC_ENGINE_NAME,
};
use store_api::region_engine::{
RegionEngineRef, RegionRole, RegionStatistic, SetRegionRoleStateResponse,
RegionEngineRef, RegionManifestInfo, RegionRole, RegionStatistic, SetRegionRoleStateResponse,
SettableRegionRoleState,
};
use store_api::region_request::{
@@ -308,6 +308,22 @@ impl RegionServer {
.with_context(|_| HandleRegionRequestSnafu { region_id })
}
pub async fn sync_region_manifest(
&self,
region_id: RegionId,
manifest_info: RegionManifestInfo,
) -> Result<()> {
let engine = self
.inner
.region_map
.get(&region_id)
.with_context(|| RegionNotFoundSnafu { region_id })?;
engine
.sync_region(region_id, manifest_info)
.await
.with_context(|_| HandleRegionRequestSnafu { region_id })
}
/// Set region role state gracefully.
///
/// For [SettableRegionRoleState::Follower]:

View File

@@ -32,8 +32,8 @@ use query::{QueryEngine, QueryEngineContext};
use session::context::QueryContextRef;
use store_api::metadata::RegionMetadataRef;
use store_api::region_engine::{
RegionEngine, RegionRole, RegionScannerRef, RegionStatistic, SetRegionRoleStateResponse,
SettableRegionRoleState,
RegionEngine, RegionManifestInfo, RegionRole, RegionScannerRef, RegionStatistic,
SetRegionRoleStateResponse, SettableRegionRoleState,
};
use store_api::region_request::{AffectedRows, RegionRequest};
use store_api::storage::{RegionId, ScanRequest, SequenceNumber};
@@ -246,6 +246,14 @@ impl RegionEngine for MockRegionEngine {
Some(RegionRole::Leader)
}
async fn sync_region(
&self,
_region_id: RegionId,
_manifest_info: RegionManifestInfo,
) -> Result<(), BoxedError> {
unimplemented!()
}
fn as_any(&self) -> &dyn Any {
self
}

View File

@@ -28,8 +28,9 @@ use snafu::{ensure, ResultExt};
use crate::error::{self, DuplicateColumnSnafu, Error, ProjectArrowSchemaSnafu, Result};
use crate::prelude::ConcreteDataType;
pub use crate::schema::column_schema::{
ColumnSchema, FulltextAnalyzer, FulltextOptions, Metadata, SkippingIndexOptions,
SkippingIndexType, COLUMN_FULLTEXT_CHANGE_OPT_KEY_ENABLE, COLUMN_FULLTEXT_OPT_KEY_ANALYZER,
ColumnSchema, FulltextAnalyzer, FulltextBackend, FulltextOptions, Metadata,
SkippingIndexOptions, SkippingIndexType, COLUMN_FULLTEXT_CHANGE_OPT_KEY_ENABLE,
COLUMN_FULLTEXT_OPT_KEY_ANALYZER, COLUMN_FULLTEXT_OPT_KEY_BACKEND,
COLUMN_FULLTEXT_OPT_KEY_CASE_SENSITIVE, COLUMN_SKIPPING_INDEX_OPT_KEY_GRANULARITY,
COLUMN_SKIPPING_INDEX_OPT_KEY_TYPE, COMMENT_KEY, FULLTEXT_KEY, INVERTED_INDEX_KEY,
SKIPPING_INDEX_KEY, TIME_INDEX_KEY,

View File

@@ -46,6 +46,7 @@ pub const SKIPPING_INDEX_KEY: &str = "greptime:skipping_index";
pub const COLUMN_FULLTEXT_CHANGE_OPT_KEY_ENABLE: &str = "enable";
pub const COLUMN_FULLTEXT_OPT_KEY_ANALYZER: &str = "analyzer";
pub const COLUMN_FULLTEXT_OPT_KEY_CASE_SENSITIVE: &str = "case_sensitive";
pub const COLUMN_FULLTEXT_OPT_KEY_BACKEND: &str = "backend";
/// Keys used in SKIPPING index options
pub const COLUMN_SKIPPING_INDEX_OPT_KEY_GRANULARITY: &str = "granularity";
@@ -514,6 +515,9 @@ pub struct FulltextOptions {
/// Whether the fulltext index is case-sensitive.
#[serde(default)]
pub case_sensitive: bool,
/// The fulltext backend to use.
#[serde(default)]
pub backend: FulltextBackend,
}
impl fmt::Display for FulltextOptions {
@@ -522,11 +526,30 @@ impl fmt::Display for FulltextOptions {
if self.enable {
write!(f, ", analyzer={}", self.analyzer)?;
write!(f, ", case_sensitive={}", self.case_sensitive)?;
write!(f, ", backend={}", self.backend)?;
}
Ok(())
}
}
/// The backend of the fulltext index.
#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize, Default, Visit, VisitMut)]
#[serde(rename_all = "kebab-case")]
pub enum FulltextBackend {
#[default]
Tantivy,
Bloom, // TODO(zhongzc): when bloom is ready, use it as default
}
impl fmt::Display for FulltextBackend {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
match self {
FulltextBackend::Tantivy => write!(f, "tantivy"),
FulltextBackend::Bloom => write!(f, "bloom"),
}
}
}
impl TryFrom<HashMap<String, String>> for FulltextOptions {
type Error = Error;
@@ -575,6 +598,19 @@ impl TryFrom<HashMap<String, String>> for FulltextOptions {
}
}
if let Some(backend) = options.get(COLUMN_FULLTEXT_OPT_KEY_BACKEND) {
match backend.to_ascii_lowercase().as_str() {
"bloom" => fulltext_options.backend = FulltextBackend::Bloom,
"tantivy" => fulltext_options.backend = FulltextBackend::Tantivy,
_ => {
return InvalidFulltextOptionSnafu {
msg: format!("{backend}, expected: 'bloom' | 'tantivy'"),
}
.fail();
}
}
}
Ok(fulltext_options)
}
}

View File

@@ -26,8 +26,8 @@ use object_store::ObjectStore;
use snafu::{ensure, OptionExt};
use store_api::metadata::RegionMetadataRef;
use store_api::region_engine::{
RegionEngine, RegionRole, RegionScannerRef, RegionStatistic, SetRegionRoleStateResponse,
SettableRegionRoleState, SinglePartitionScanner,
RegionEngine, RegionManifestInfo, RegionRole, RegionScannerRef, RegionStatistic,
SetRegionRoleStateResponse, SettableRegionRoleState, SinglePartitionScanner,
};
use store_api::region_request::{
AffectedRows, RegionCloseRequest, RegionCreateRequest, RegionDropRequest, RegionOpenRequest,
@@ -138,6 +138,15 @@ impl RegionEngine for FileRegionEngine {
}
}
async fn sync_region(
&self,
_region_id: RegionId,
_manifest_info: RegionManifestInfo,
) -> Result<(), BoxedError> {
// File engine doesn't need to sync region manifest.
Ok(())
}
fn role(&self, region_id: RegionId) -> Option<RegionRole> {
self.inner.state(region_id)
}

View File

@@ -37,7 +37,6 @@ use serde::{Deserialize, Serialize};
use servers::grpc::GrpcOptions;
use servers::heartbeat_options::HeartbeatOptions;
use servers::http::HttpOptions;
use servers::Mode;
use session::context::QueryContext;
use snafu::{ensure, OptionExt, ResultExt};
use store_api::storage::{ConcreteDataType, RegionId};
@@ -63,7 +62,7 @@ pub(crate) mod refill;
mod stat;
#[cfg(test)]
mod tests;
mod util;
pub(crate) mod util;
mod worker;
pub(crate) mod node_context;
@@ -102,7 +101,6 @@ impl Default for FlowConfig {
#[derive(Clone, Debug, Serialize, Deserialize)]
#[serde(default)]
pub struct FlownodeOptions {
pub mode: Mode,
pub node_id: Option<u64>,
pub flow: FlowConfig,
pub grpc: GrpcOptions,
@@ -116,7 +114,6 @@ pub struct FlownodeOptions {
impl Default for FlownodeOptions {
fn default() -> Self {
Self {
mode: servers::Mode::Standalone,
node_id: None,
flow: FlowConfig::default(),
grpc: GrpcOptions::default().with_bind_addr("127.0.0.1:3004"),

View File

@@ -12,6 +12,8 @@
// See the License for the specific language governing permissions and
// limitations under the License.
//! Util functions for adapter
use std::sync::Arc;
use api::helper::ColumnDataTypeWrapper;

View File

@@ -16,6 +16,7 @@
use std::any::Any;
use arrow_schema::ArrowError;
use common_error::ext::BoxedError;
use common_error::{define_into_tonic_status, from_err_code_msg_to_header};
use common_macro::stack_trace_debug;
@@ -156,6 +157,15 @@ pub enum Error {
location: Location,
},
#[snafu(display("Arrow error: {raw:?} in context: {context}"))]
Arrow {
#[snafu(source)]
raw: ArrowError,
context: String,
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Datafusion error: {raw:?} in context: {context}"))]
Datafusion {
#[snafu(source)]
@@ -238,7 +248,9 @@ impl ErrorExt for Error {
| Self::FlowNotFound { .. }
| Self::ListFlows { .. } => StatusCode::TableNotFound,
Self::Plan { .. } | Self::Datatypes { .. } => StatusCode::PlanQuery,
Self::InvalidQuery { .. } | Self::CreateFlow { .. } => StatusCode::EngineExecuteQuery,
Self::InvalidQuery { .. } | Self::CreateFlow { .. } | Self::Arrow { .. } => {
StatusCode::EngineExecuteQuery
}
Self::Unexpected { .. } => StatusCode::Unexpected,
Self::NotImplemented { .. } | Self::UnsupportedTemporalFilter { .. } => {
StatusCode::Unsupported

View File

@@ -17,6 +17,7 @@
use std::time::Duration;
mod frontend_client;
mod time_window;
mod utils;
/// TODO(discord9): make those constants configurable

View File

@@ -0,0 +1,839 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
//! Time window expr and helper functions
//!
use std::collections::BTreeSet;
use std::sync::Arc;
use api::helper::pb_value_to_value_ref;
use arrow::array::{
TimestampMicrosecondArray, TimestampMillisecondArray, TimestampNanosecondArray,
TimestampSecondArray,
};
use catalog::CatalogManagerRef;
use common_error::ext::BoxedError;
use common_recordbatch::DfRecordBatch;
use common_telemetry::warn;
use common_time::timestamp::TimeUnit;
use common_time::Timestamp;
use datafusion::error::Result as DfResult;
use datafusion::execution::SessionState;
use datafusion::logical_expr::Expr;
use datafusion::physical_planner::{DefaultPhysicalPlanner, PhysicalPlanner};
use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRecursion, TreeNodeRewriter};
use datafusion_common::{DFSchema, TableReference};
use datafusion_expr::{ColumnarValue, LogicalPlan};
use datafusion_physical_expr::PhysicalExprRef;
use datatypes::prelude::{ConcreteDataType, DataType};
use datatypes::schema::TIME_INDEX_KEY;
use datatypes::value::Value;
use datatypes::vectors::{
TimestampMicrosecondVector, TimestampMillisecondVector, TimestampNanosecondVector,
TimestampSecondVector, Vector,
};
use itertools::Itertools;
use session::context::QueryContextRef;
use snafu::{ensure, OptionExt, ResultExt};
use crate::adapter::util::from_proto_to_data_type;
use crate::error::{
ArrowSnafu, DatafusionSnafu, DatatypesSnafu, ExternalSnafu, PlanSnafu, UnexpectedSnafu,
};
use crate::expr::error::DataTypeSnafu;
use crate::Error;
/// Time window expr like `date_bin(INTERVAL '1' MINUTE, ts)`, this type help with
/// evaluating the expr using given timestamp
///
/// The time window expr must satisfies following conditions:
/// 1. The expr must be monotonic non-decreasing
/// 2. The expr must only have one and only one input column with timestamp type, and the output column must be timestamp type
/// 3. The expr must be deterministic
///
/// An example of time window expr is `date_bin(INTERVAL '1' MINUTE, ts)`
#[derive(Debug, Clone)]
pub struct TimeWindowExpr {
phy_expr: PhysicalExprRef,
column_name: String,
logical_expr: Expr,
df_schema: DFSchema,
}
impl TimeWindowExpr {
pub fn from_expr(
expr: &Expr,
column_name: &str,
df_schema: &DFSchema,
session: &SessionState,
) -> Result<Self, Error> {
let phy_expr: PhysicalExprRef = to_phy_expr(expr, df_schema, session)?;
Ok(Self {
phy_expr,
column_name: column_name.to_string(),
logical_expr: expr.clone(),
df_schema: df_schema.clone(),
})
}
pub fn eval(
&self,
current: Timestamp,
) -> Result<(Option<Timestamp>, Option<Timestamp>), Error> {
let lower_bound =
calc_expr_time_window_lower_bound(&self.phy_expr, &self.df_schema, current)?;
let upper_bound =
probe_expr_time_window_upper_bound(&self.phy_expr, &self.df_schema, current)?;
Ok((lower_bound, upper_bound))
}
/// Find timestamps from rows using time window expr
///
/// use column of name `self.column_name` from input rows list as input to time window expr
pub async fn handle_rows(
&self,
rows_list: Vec<api::v1::Rows>,
) -> Result<BTreeSet<Timestamp>, Error> {
let mut time_windows = BTreeSet::new();
for rows in rows_list {
// pick the time index column and use it to eval on `self.expr`
// TODO(discord9): handle case where time index column is not present(i.e. DEFAULT constant value)
let ts_col_index = rows
.schema
.iter()
.map(|col| col.column_name.clone())
.position(|name| name == self.column_name);
let Some(ts_col_index) = ts_col_index else {
warn!("can't found time index column in schema: {:?}", rows.schema);
continue;
};
let col_schema = &rows.schema[ts_col_index];
let cdt = from_proto_to_data_type(col_schema)?;
let mut vector = cdt.create_mutable_vector(rows.rows.len());
for row in rows.rows {
let value = pb_value_to_value_ref(&row.values[ts_col_index], &None);
vector.try_push_value_ref(value).context(DataTypeSnafu {
msg: "Failed to convert rows to columns",
})?;
}
let vector = vector.to_vector();
let df_schema = create_df_schema_for_ts_column(&self.column_name, cdt)?;
let rb =
DfRecordBatch::try_new(df_schema.inner().clone(), vec![vector.to_arrow_array()])
.with_context(|_e| ArrowSnafu {
context: format!(
"Failed to create record batch from {df_schema:?} and {vector:?}"
),
})?;
let eval_res = self
.phy_expr
.evaluate(&rb)
.with_context(|_| DatafusionSnafu {
context: format!(
"Failed to evaluate physical expression {:?} on {rb:?}",
self.phy_expr
),
})?;
let res = columnar_to_ts_vector(&eval_res)?;
for ts in res.into_iter().flatten() {
time_windows.insert(ts);
}
}
Ok(time_windows)
}
}
fn create_df_schema_for_ts_column(name: &str, cdt: ConcreteDataType) -> Result<DFSchema, Error> {
let arrow_schema = Arc::new(arrow_schema::Schema::new(vec![arrow_schema::Field::new(
name,
cdt.as_arrow_type(),
false,
)]));
let df_schema = DFSchema::from_field_specific_qualified_schema(
vec![Some(TableReference::bare("TimeIndexOnlyTable"))],
&arrow_schema,
)
.with_context(|_e| DatafusionSnafu {
context: format!("Failed to create DFSchema from arrow schema {arrow_schema:?}"),
})?;
Ok(df_schema)
}
/// Convert `ColumnarValue` to `Vec<Option<Timestamp>>`
fn columnar_to_ts_vector(columnar: &ColumnarValue) -> Result<Vec<Option<Timestamp>>, Error> {
let val = match columnar {
datafusion_expr::ColumnarValue::Array(array) => {
let ty = array.data_type();
let ty = ConcreteDataType::from_arrow_type(ty);
let time_unit = if let ConcreteDataType::Timestamp(ty) = ty {
ty.unit()
} else {
return UnexpectedSnafu {
reason: format!("Non-timestamp type: {ty:?}"),
}
.fail();
};
match time_unit {
TimeUnit::Second => array
.as_ref()
.as_any()
.downcast_ref::<TimestampSecondArray>()
.with_context(|| PlanSnafu {
reason: format!("Failed to create vector from arrow array {array:?}"),
})?
.values()
.iter()
.map(|d| Some(Timestamp::new(*d, time_unit)))
.collect_vec(),
TimeUnit::Millisecond => array
.as_ref()
.as_any()
.downcast_ref::<TimestampMillisecondArray>()
.with_context(|| PlanSnafu {
reason: format!("Failed to create vector from arrow array {array:?}"),
})?
.values()
.iter()
.map(|d| Some(Timestamp::new(*d, time_unit)))
.collect_vec(),
TimeUnit::Microsecond => array
.as_ref()
.as_any()
.downcast_ref::<TimestampMicrosecondArray>()
.with_context(|| PlanSnafu {
reason: format!("Failed to create vector from arrow array {array:?}"),
})?
.values()
.iter()
.map(|d| Some(Timestamp::new(*d, time_unit)))
.collect_vec(),
TimeUnit::Nanosecond => array
.as_ref()
.as_any()
.downcast_ref::<TimestampNanosecondArray>()
.with_context(|| PlanSnafu {
reason: format!("Failed to create vector from arrow array {array:?}"),
})?
.values()
.iter()
.map(|d| Some(Timestamp::new(*d, time_unit)))
.collect_vec(),
}
}
datafusion_expr::ColumnarValue::Scalar(scalar) => {
let value = Value::try_from(scalar.clone()).with_context(|_| DatatypesSnafu {
extra: format!("Failed to convert scalar {scalar:?} to value"),
})?;
let ts = value.as_timestamp().context(UnexpectedSnafu {
reason: format!("Expect Timestamp, found {:?}", value),
})?;
vec![Some(ts)]
}
};
Ok(val)
}
/// Return (the column name of time index column, the time window expr, the expected time unit of time index column, the expr's schema for evaluating the time window)
///
/// The time window expr is expected to have one input column with Timestamp type, and also return Timestamp type, the time window expr is expected
/// to be monotonic increasing and appears in the innermost GROUP BY clause
///
/// note this plan should only contain one TableScan
async fn find_time_window_expr(
plan: &LogicalPlan,
catalog_man: CatalogManagerRef,
query_ctx: QueryContextRef,
) -> Result<(String, Option<datafusion_expr::Expr>, TimeUnit, DFSchema), Error> {
// TODO(discord9): find the expr that do time window
let mut table_name = None;
// first find the table source in the logical plan
plan.apply(|plan| {
let LogicalPlan::TableScan(table_scan) = plan else {
return Ok(TreeNodeRecursion::Continue);
};
table_name = Some(table_scan.table_name.clone());
Ok(TreeNodeRecursion::Stop)
})
.with_context(|_| DatafusionSnafu {
context: format!("Can't find table source in plan {plan:?}"),
})?;
let Some(table_name) = table_name else {
UnexpectedSnafu {
reason: format!("Can't find table source in plan {plan:?}"),
}
.fail()?
};
let current_schema = query_ctx.current_schema();
let catalog_name = table_name.catalog().unwrap_or(query_ctx.current_catalog());
let schema_name = table_name.schema().unwrap_or(&current_schema);
let table_name = table_name.table();
let Some(table_ref) = catalog_man
.table(catalog_name, schema_name, table_name, Some(&query_ctx))
.await
.map_err(BoxedError::new)
.context(ExternalSnafu)?
else {
UnexpectedSnafu {
reason: format!(
"Can't find table {table_name:?} in catalog {catalog_name:?}/{schema_name:?}"
),
}
.fail()?
};
let schema = &table_ref.table_info().meta.schema;
let ts_index = schema.timestamp_column().with_context(|| UnexpectedSnafu {
reason: format!("Can't find timestamp column in table {table_name:?}"),
})?;
let ts_col_name = ts_index.name.clone();
let expected_time_unit = ts_index.data_type.as_timestamp().with_context(|| UnexpectedSnafu {
reason: format!(
"Expected timestamp column {ts_col_name:?} in table {table_name:?} to be timestamp, but got {ts_index:?}"
),
})?.unit();
let arrow_schema = Arc::new(arrow_schema::Schema::new(vec![arrow_schema::Field::new(
ts_col_name.clone(),
ts_index.data_type.as_arrow_type(),
false,
)]));
let df_schema = DFSchema::from_field_specific_qualified_schema(
vec![Some(TableReference::bare(table_name))],
&arrow_schema,
)
.with_context(|_e| DatafusionSnafu {
context: format!("Failed to create DFSchema from arrow schema {arrow_schema:?}"),
})?;
// find the time window expr which refers to the time index column
let mut aggr_expr = None;
let mut time_window_expr: Option<Expr> = None;
let find_inner_aggr_expr = |plan: &LogicalPlan| {
if let LogicalPlan::Aggregate(aggregate) = plan {
aggr_expr = Some(aggregate.clone());
};
Ok(TreeNodeRecursion::Continue)
};
plan.apply(find_inner_aggr_expr)
.with_context(|_| DatafusionSnafu {
context: format!("Can't find aggr expr in plan {plan:?}"),
})?;
if let Some(aggregate) = aggr_expr {
for group_expr in &aggregate.group_expr {
let refs = group_expr.column_refs();
if refs.len() != 1 {
continue;
}
let ref_col = refs.iter().next().unwrap();
let index = aggregate.input.schema().maybe_index_of_column(ref_col);
let Some(index) = index else {
continue;
};
let field = aggregate.input.schema().field(index);
// TODO(discord9): need to ensure the field has the meta key for the time index
let is_time_index =
field.metadata().get(TIME_INDEX_KEY).map(|s| s.as_str()) == Some("true");
if is_time_index {
let rewrite_column = group_expr.clone();
let rewritten = rewrite_column
.rewrite(&mut RewriteColumn {
table_name: table_name.to_string(),
})
.with_context(|_| DatafusionSnafu {
context: format!("Rewrite expr failed, expr={:?}", group_expr),
})?
.data;
struct RewriteColumn {
table_name: String,
}
impl TreeNodeRewriter for RewriteColumn {
type Node = Expr;
fn f_down(&mut self, node: Self::Node) -> DfResult<Transformed<Self::Node>> {
let Expr::Column(mut column) = node else {
return Ok(Transformed::no(node));
};
column.relation = Some(TableReference::bare(self.table_name.clone()));
Ok(Transformed::yes(Expr::Column(column)))
}
}
time_window_expr = Some(rewritten);
break;
}
}
Ok((ts_col_name, time_window_expr, expected_time_unit, df_schema))
} else {
// can't found time window expr, return None
Ok((ts_col_name, None, expected_time_unit, df_schema))
}
}
/// Find nearest lower bound for time `current` in given `plan` for the time window expr.
/// i.e. for time window expr being `date_bin(INTERVAL '5 minutes', ts) as time_window` and `current="2021-07-01 00:01:01.000"`,
/// return `Some("2021-07-01 00:00:00.000")`
/// if `plan` doesn't contain a `TIME INDEX` column, return `None`
///
/// Time window expr is a expr that:
/// 1. ref only to a time index column
/// 2. is monotonic increasing
/// 3. show up in GROUP BY clause
///
/// note this plan should only contain one TableScan
#[cfg(test)]
pub async fn find_plan_time_window_bound(
plan: &LogicalPlan,
current: Timestamp,
query_ctx: QueryContextRef,
engine: query::QueryEngineRef,
) -> Result<(String, Option<Timestamp>, Option<Timestamp>), Error> {
// TODO(discord9): find the expr that do time window
let catalog_man = engine.engine_state().catalog_manager();
let (ts_col_name, time_window_expr, expected_time_unit, df_schema) =
find_time_window_expr(plan, catalog_man.clone(), query_ctx).await?;
// cast current to ts_index's type
let new_current = current
.convert_to(expected_time_unit)
.with_context(|| UnexpectedSnafu {
reason: format!("Failed to cast current timestamp {current:?} to {expected_time_unit}"),
})?;
// if no time_window_expr is found, return None
if let Some(time_window_expr) = time_window_expr {
let phy_expr = to_phy_expr(
&time_window_expr,
&df_schema,
&engine.engine_state().session_state(),
)?;
let lower_bound = calc_expr_time_window_lower_bound(&phy_expr, &df_schema, new_current)?;
let upper_bound = probe_expr_time_window_upper_bound(&phy_expr, &df_schema, new_current)?;
Ok((ts_col_name, lower_bound, upper_bound))
} else {
Ok((ts_col_name, None, None))
}
}
/// Find the lower bound of time window in given `expr` and `current` timestamp.
///
/// i.e. for `current="2021-07-01 00:01:01.000"` and `expr=date_bin(INTERVAL '5 minutes', ts) as time_window` and `ts_col=ts`,
/// return `Some("2021-07-01 00:00:00.000")` since it's the lower bound
/// return `Some("2021-07-01 00:00:00.000")` since it's the lower bound
/// of current time window given the current timestamp
///
/// if return None, meaning this time window have no lower bound
fn calc_expr_time_window_lower_bound(
phy_expr: &PhysicalExprRef,
df_schema: &DFSchema,
current: Timestamp,
) -> Result<Option<Timestamp>, Error> {
let cur_time_window = eval_phy_time_window_expr(phy_expr, df_schema, current)?;
let input_time_unit = cur_time_window.unit();
Ok(cur_time_window.convert_to(input_time_unit))
}
/// Probe for the upper bound for time window expression
fn probe_expr_time_window_upper_bound(
phy_expr: &PhysicalExprRef,
df_schema: &DFSchema,
current: Timestamp,
) -> Result<Option<Timestamp>, Error> {
// TODO(discord9): special handling `date_bin` for faster path
use std::cmp::Ordering;
let cur_time_window = eval_phy_time_window_expr(phy_expr, df_schema, current)?;
// search to find the lower bound
let mut offset: i64 = 1;
let mut lower_bound = Some(current);
let upper_bound;
// first expontial probe to found a range for binary search
loop {
let Some(next_val) = current.value().checked_add(offset) else {
// no upper bound if overflow, which is ok
return Ok(None);
};
let next_time_probe = common_time::Timestamp::new(next_val, current.unit());
let next_time_window = eval_phy_time_window_expr(phy_expr, df_schema, next_time_probe)?;
match next_time_window.cmp(&cur_time_window) {
Ordering::Less => UnexpectedSnafu {
reason: format!(
"Unsupported time window expression, expect monotonic increasing for time window expression {phy_expr:?}"
),
}
.fail()?,
Ordering::Equal => {
lower_bound = Some(next_time_probe);
}
Ordering::Greater => {
upper_bound = Some(next_time_probe);
break
}
}
let Some(new_offset) = offset.checked_mul(2) else {
// no upper bound if overflow
return Ok(None);
};
offset = new_offset;
}
// binary search for the exact upper bound
binary_search_expr(
lower_bound,
upper_bound,
cur_time_window,
phy_expr,
df_schema,
)
.map(Some)
}
fn binary_search_expr(
lower_bound: Option<Timestamp>,
upper_bound: Option<Timestamp>,
cur_time_window: Timestamp,
phy_expr: &PhysicalExprRef,
df_schema: &DFSchema,
) -> Result<Timestamp, Error> {
ensure!(lower_bound.map(|v|v.unit()) == upper_bound.map(|v| v.unit()), UnexpectedSnafu {
reason: format!(" unit mismatch for time window expression {phy_expr:?}, found {lower_bound:?} and {upper_bound:?}"),
});
let output_unit = upper_bound
.context(UnexpectedSnafu {
reason: "should have lower bound",
})?
.unit();
let mut low = lower_bound
.context(UnexpectedSnafu {
reason: "should have lower bound",
})?
.value();
let mut high = upper_bound
.context(UnexpectedSnafu {
reason: "should have upper bound",
})?
.value();
while low < high {
let mid = (low + high) / 2;
let mid_probe = common_time::Timestamp::new(mid, output_unit);
let mid_time_window = eval_phy_time_window_expr(phy_expr, df_schema, mid_probe)?;
match mid_time_window.cmp(&cur_time_window) {
std::cmp::Ordering::Less => UnexpectedSnafu {
reason: format!("Binary search failed for time window expression {phy_expr:?}"),
}
.fail()?,
std::cmp::Ordering::Equal => low = mid + 1,
std::cmp::Ordering::Greater => high = mid,
}
}
let final_upper_bound_for_time_window = common_time::Timestamp::new(high, output_unit);
Ok(final_upper_bound_for_time_window)
}
/// Expect the `phy` expression only have one input column with Timestamp type, and also return Timestamp type
fn eval_phy_time_window_expr(
phy: &PhysicalExprRef,
df_schema: &DFSchema,
input_value: Timestamp,
) -> Result<Timestamp, Error> {
let schema_ty = df_schema.field(0).data_type();
let schema_cdt = ConcreteDataType::from_arrow_type(schema_ty);
let schema_unit = if let ConcreteDataType::Timestamp(ts) = schema_cdt {
ts.unit()
} else {
return UnexpectedSnafu {
reason: format!("Expect Timestamp, found {:?}", schema_cdt),
}
.fail();
};
let input_value = input_value
.convert_to(schema_unit)
.with_context(|| UnexpectedSnafu {
reason: format!("Failed to convert timestamp {input_value:?} to {schema_unit}"),
})?;
let ts_vector = match schema_unit {
TimeUnit::Second => {
TimestampSecondVector::from_vec(vec![input_value.value()]).to_arrow_array()
}
TimeUnit::Millisecond => {
TimestampMillisecondVector::from_vec(vec![input_value.value()]).to_arrow_array()
}
TimeUnit::Microsecond => {
TimestampMicrosecondVector::from_vec(vec![input_value.value()]).to_arrow_array()
}
TimeUnit::Nanosecond => {
TimestampNanosecondVector::from_vec(vec![input_value.value()]).to_arrow_array()
}
};
let rb = DfRecordBatch::try_new(df_schema.inner().clone(), vec![ts_vector.clone()])
.with_context(|_| ArrowSnafu {
context: format!("Failed to create record batch from {df_schema:?} and {ts_vector:?}"),
})?;
let eval_res = phy.evaluate(&rb).with_context(|_| DatafusionSnafu {
context: format!("Failed to evaluate physical expression {phy:?} on {rb:?}"),
})?;
if let Some(Some(ts)) = columnar_to_ts_vector(&eval_res)?.first() {
Ok(*ts)
} else {
UnexpectedSnafu {
reason: format!(
"Expected timestamp in expression {phy:?} but got {:?}",
eval_res
),
}
.fail()?
}
}
fn to_phy_expr(
expr: &Expr,
df_schema: &DFSchema,
session: &SessionState,
) -> Result<PhysicalExprRef, Error> {
let phy_planner = DefaultPhysicalPlanner::default();
let phy_expr: PhysicalExprRef = phy_planner
.create_physical_expr(expr, df_schema, session)
.with_context(|_e| DatafusionSnafu {
context: format!(
"Failed to create physical expression from {expr:?} using {df_schema:?}"
),
})?;
Ok(phy_expr)
}
#[cfg(test)]
mod test {
use datafusion_common::tree_node::TreeNode;
use pretty_assertions::assert_eq;
use session::context::QueryContext;
use super::*;
use crate::recording_rules::utils::{df_plan_to_sql, sql_to_df_plan, AddFilterRewriter};
use crate::test_utils::create_test_query_engine;
#[tokio::test]
async fn test_plan_time_window_lower_bound() {
use datafusion_expr::{col, lit};
let query_engine = create_test_query_engine();
let ctx = QueryContext::arc();
let testcases = [
// same alias is not same column
(
"SELECT arrow_cast(date_bin(INTERVAL '1 MINS', numbers_with_ts.ts), 'Timestamp(Second, None)') AS ts FROM numbers_with_ts GROUP BY ts;",
Timestamp::new(1740394109, TimeUnit::Second),
(
"ts".to_string(),
Some(Timestamp::new(1740394109000, TimeUnit::Millisecond)),
Some(Timestamp::new(1740394109001, TimeUnit::Millisecond)),
),
r#"SELECT arrow_cast(date_bin(INTERVAL '1 MINS', numbers_with_ts.ts), 'Timestamp(Second, None)') AS ts FROM numbers_with_ts WHERE ((ts >= CAST('2025-02-24 10:48:29' AS TIMESTAMP)) AND (ts <= CAST('2025-02-24 10:48:29.001' AS TIMESTAMP))) GROUP BY numbers_with_ts.ts"#
),
// complex time window index
(
"SELECT arrow_cast(date_bin(INTERVAL '1 MINS', numbers_with_ts.ts), 'Timestamp(Second, None)') AS time_window FROM numbers_with_ts GROUP BY time_window;",
Timestamp::new(1740394109, TimeUnit::Second),
(
"ts".to_string(),
Some(Timestamp::new(1740394080, TimeUnit::Second)),
Some(Timestamp::new(1740394140, TimeUnit::Second)),
),
"SELECT arrow_cast(date_bin(INTERVAL '1 MINS', numbers_with_ts.ts), 'Timestamp(Second, None)') AS time_window FROM numbers_with_ts WHERE ((ts >= CAST('2025-02-24 10:48:00' AS TIMESTAMP)) AND (ts <= CAST('2025-02-24 10:49:00' AS TIMESTAMP))) GROUP BY arrow_cast(date_bin(INTERVAL '1 MINS', numbers_with_ts.ts), 'Timestamp(Second, None)')"
),
// no time index
(
"SELECT date_bin('5 minutes', ts) FROM numbers_with_ts;",
Timestamp::new(23, TimeUnit::Millisecond),
("ts".to_string(), None, None),
"SELECT date_bin('5 minutes', ts) FROM numbers_with_ts;"
),
// time index
(
"SELECT date_bin('5 minutes', ts) as time_window FROM numbers_with_ts GROUP BY time_window;",
Timestamp::new(23, TimeUnit::Nanosecond),
(
"ts".to_string(),
Some(Timestamp::new(0, TimeUnit::Millisecond)),
Some(Timestamp::new(300000, TimeUnit::Millisecond)),
),
"SELECT date_bin('5 minutes', numbers_with_ts.ts) AS time_window FROM numbers_with_ts WHERE ((ts >= CAST('1970-01-01 00:00:00' AS TIMESTAMP)) AND (ts <= CAST('1970-01-01 00:05:00' AS TIMESTAMP))) GROUP BY date_bin('5 minutes', numbers_with_ts.ts)"
),
// on spot
(
"SELECT date_bin('5 minutes', ts) as time_window FROM numbers_with_ts GROUP BY time_window;",
Timestamp::new(0, TimeUnit::Nanosecond),
(
"ts".to_string(),
Some(Timestamp::new(0, TimeUnit::Millisecond)),
Some(Timestamp::new(300000, TimeUnit::Millisecond)),
),
"SELECT date_bin('5 minutes', numbers_with_ts.ts) AS time_window FROM numbers_with_ts WHERE ((ts >= CAST('1970-01-01 00:00:00' AS TIMESTAMP)) AND (ts <= CAST('1970-01-01 00:05:00' AS TIMESTAMP))) GROUP BY date_bin('5 minutes', numbers_with_ts.ts)"
),
// different time unit
(
"SELECT date_bin('5 minutes', ts) as time_window FROM numbers_with_ts GROUP BY time_window;",
Timestamp::new(23_000_000, TimeUnit::Nanosecond),
(
"ts".to_string(),
Some(Timestamp::new(0, TimeUnit::Millisecond)),
Some(Timestamp::new(300000, TimeUnit::Millisecond)),
),
"SELECT date_bin('5 minutes', numbers_with_ts.ts) AS time_window FROM numbers_with_ts WHERE ((ts >= CAST('1970-01-01 00:00:00' AS TIMESTAMP)) AND (ts <= CAST('1970-01-01 00:05:00' AS TIMESTAMP))) GROUP BY date_bin('5 minutes', numbers_with_ts.ts)"
),
// time index with other fields
(
"SELECT sum(number) as sum_up, date_bin('5 minutes', ts) as time_window FROM numbers_with_ts GROUP BY time_window;",
Timestamp::new(23, TimeUnit::Millisecond),
(
"ts".to_string(),
Some(Timestamp::new(0, TimeUnit::Millisecond)),
Some(Timestamp::new(300000, TimeUnit::Millisecond)),
),
"SELECT sum(numbers_with_ts.number) AS sum_up, date_bin('5 minutes', numbers_with_ts.ts) AS time_window FROM numbers_with_ts WHERE ((ts >= CAST('1970-01-01 00:00:00' AS TIMESTAMP)) AND (ts <= CAST('1970-01-01 00:05:00' AS TIMESTAMP))) GROUP BY date_bin('5 minutes', numbers_with_ts.ts)"
),
// time index with other pks
(
"SELECT number, date_bin('5 minutes', ts) as time_window FROM numbers_with_ts GROUP BY time_window, number;",
Timestamp::new(23, TimeUnit::Millisecond),
(
"ts".to_string(),
Some(Timestamp::new(0, TimeUnit::Millisecond)),
Some(Timestamp::new(300000, TimeUnit::Millisecond)),
),
"SELECT numbers_with_ts.number, date_bin('5 minutes', numbers_with_ts.ts) AS time_window FROM numbers_with_ts WHERE ((ts >= CAST('1970-01-01 00:00:00' AS TIMESTAMP)) AND (ts <= CAST('1970-01-01 00:05:00' AS TIMESTAMP))) GROUP BY date_bin('5 minutes', numbers_with_ts.ts), numbers_with_ts.number"
),
// subquery
(
"SELECT number, time_window FROM (SELECT number, date_bin('5 minutes', ts) as time_window FROM numbers_with_ts GROUP BY time_window, number);",
Timestamp::new(23, TimeUnit::Millisecond),
(
"ts".to_string(),
Some(Timestamp::new(0, TimeUnit::Millisecond)),
Some(Timestamp::new(300000, TimeUnit::Millisecond)),
),
"SELECT numbers_with_ts.number, time_window FROM (SELECT numbers_with_ts.number, date_bin('5 minutes', numbers_with_ts.ts) AS time_window FROM numbers_with_ts WHERE ((ts >= CAST('1970-01-01 00:00:00' AS TIMESTAMP)) AND (ts <= CAST('1970-01-01 00:05:00' AS TIMESTAMP))) GROUP BY date_bin('5 minutes', numbers_with_ts.ts), numbers_with_ts.number)"
),
// cte
(
"with cte as (select number, date_bin('5 minutes', ts) as time_window from numbers_with_ts GROUP BY time_window, number) select number, time_window from cte;",
Timestamp::new(23, TimeUnit::Millisecond),
(
"ts".to_string(),
Some(Timestamp::new(0, TimeUnit::Millisecond)),
Some(Timestamp::new(300000, TimeUnit::Millisecond)),
),
"SELECT cte.number, cte.time_window FROM (SELECT numbers_with_ts.number, date_bin('5 minutes', numbers_with_ts.ts) AS time_window FROM numbers_with_ts WHERE ((ts >= CAST('1970-01-01 00:00:00' AS TIMESTAMP)) AND (ts <= CAST('1970-01-01 00:05:00' AS TIMESTAMP))) GROUP BY date_bin('5 minutes', numbers_with_ts.ts), numbers_with_ts.number) AS cte"
),
// complex subquery without alias
(
"SELECT sum(number), number, date_bin('5 minutes', ts) as time_window, bucket_name FROM (SELECT number, ts, case when number < 5 THEN 'bucket_0_5' when number >= 5 THEN 'bucket_5_inf' END as bucket_name FROM numbers_with_ts) GROUP BY number, time_window, bucket_name;",
Timestamp::new(23, TimeUnit::Millisecond),
(
"ts".to_string(),
Some(Timestamp::new(0, TimeUnit::Millisecond)),
Some(Timestamp::new(300000, TimeUnit::Millisecond)),
),
"SELECT sum(numbers_with_ts.number), numbers_with_ts.number, date_bin('5 minutes', numbers_with_ts.ts) AS time_window, bucket_name FROM (SELECT numbers_with_ts.number, numbers_with_ts.ts, CASE WHEN (numbers_with_ts.number < 5) THEN 'bucket_0_5' WHEN (numbers_with_ts.number >= 5) THEN 'bucket_5_inf' END AS bucket_name FROM numbers_with_ts WHERE ((ts >= CAST('1970-01-01 00:00:00' AS TIMESTAMP)) AND (ts <= CAST('1970-01-01 00:05:00' AS TIMESTAMP)))) GROUP BY numbers_with_ts.number, date_bin('5 minutes', numbers_with_ts.ts), bucket_name"
),
// complex subquery alias
(
"SELECT sum(number), number, date_bin('5 minutes', ts) as time_window, bucket_name FROM (SELECT number, ts, case when number < 5 THEN 'bucket_0_5' when number >= 5 THEN 'bucket_5_inf' END as bucket_name FROM numbers_with_ts) as cte GROUP BY number, time_window, bucket_name;",
Timestamp::new(23, TimeUnit::Millisecond),
(
"ts".to_string(),
Some(Timestamp::new(0, TimeUnit::Millisecond)),
Some(Timestamp::new(300000, TimeUnit::Millisecond)),
),
"SELECT sum(cte.number), cte.number, date_bin('5 minutes', cte.ts) AS time_window, cte.bucket_name FROM (SELECT numbers_with_ts.number, numbers_with_ts.ts, CASE WHEN (numbers_with_ts.number < 5) THEN 'bucket_0_5' WHEN (numbers_with_ts.number >= 5) THEN 'bucket_5_inf' END AS bucket_name FROM numbers_with_ts WHERE ((ts >= CAST('1970-01-01 00:00:00' AS TIMESTAMP)) AND (ts <= CAST('1970-01-01 00:05:00' AS TIMESTAMP)))) AS cte GROUP BY cte.number, date_bin('5 minutes', cte.ts), cte.bucket_name"
),
];
for (sql, current, expected, expected_unparsed) in testcases {
let plan = sql_to_df_plan(ctx.clone(), query_engine.clone(), sql, true)
.await
.unwrap();
let real =
find_plan_time_window_bound(&plan, current, ctx.clone(), query_engine.clone())
.await
.unwrap();
assert_eq!(expected, real);
let plan = sql_to_df_plan(ctx.clone(), query_engine.clone(), sql, false)
.await
.unwrap();
let (col_name, lower, upper) = real;
let new_sql = if lower.is_some() {
let to_df_literal = |value| {
let value = Value::from(value);
value.try_to_scalar_value(&value.data_type()).unwrap()
};
let lower = to_df_literal(lower.unwrap());
let upper = to_df_literal(upper.unwrap());
let expr = col(&col_name)
.gt_eq(lit(lower))
.and(col(&col_name).lt_eq(lit(upper)));
let mut add_filter = AddFilterRewriter::new(expr);
let plan = plan.rewrite(&mut add_filter).unwrap().data;
df_plan_to_sql(&plan).unwrap()
} else {
sql.to_string()
};
assert_eq!(expected_unparsed, new_sql);
}
}
}

View File

@@ -264,7 +264,7 @@ pub struct AddFilterRewriter {
}
impl AddFilterRewriter {
fn new(filter: Expr) -> Self {
pub fn new(filter: Expr) -> Self {
Self {
extra_filter: filter,
is_rewritten: false,

View File

@@ -19,7 +19,9 @@ use common_error::define_into_tonic_status;
use common_error::ext::{BoxedError, ErrorExt};
use common_error::status_code::StatusCode;
use common_macro::stack_trace_debug;
use session::ReadPreference;
use snafu::{Location, Snafu};
use store_api::storage::RegionId;
#[derive(Snafu)]
#[snafu(visibility(pub))]
@@ -140,9 +142,14 @@ pub enum Error {
location: Location,
},
#[snafu(display("Failed to find table route for table id {}", table_id))]
FindTableRoute {
table_id: u32,
#[snafu(display(
"Failed to find region peer for region id {}, read preference: {}",
region_id,
read_preference
))]
FindRegionPeer {
region_id: RegionId,
read_preference: ReadPreference,
#[snafu(implicit)]
location: Location,
source: partition::error::Error,
@@ -410,7 +417,7 @@ impl ErrorExt for Error {
Error::External { source, .. } | Error::InitPlugin { source, .. } => {
source.status_code()
}
Error::FindTableRoute { source, .. } => source.status_code(),
Error::FindRegionPeer { source, .. } => source.status_code(),
Error::TableOperation { source, .. } => source.status_code(),

View File

@@ -113,14 +113,19 @@ impl Instance {
.context(error::OpenRaftEngineBackendSnafu)?;
let kv_backend = Arc::new(kv_backend);
let state_store = Arc::new(KvStateStore::new(kv_backend.clone()));
let kv_state_store = Arc::new(KvStateStore::new(kv_backend.clone()));
let manager_config = ManagerConfig {
max_retry_times: procedure_config.max_retry_times,
retry_delay: procedure_config.retry_delay,
max_running_procedures: procedure_config.max_running_procedures,
..Default::default()
};
let procedure_manager = Arc::new(LocalManager::new(manager_config, state_store));
let procedure_manager = Arc::new(LocalManager::new(
manager_config,
kv_state_store.clone(),
kv_state_store,
));
Ok((kv_backend, procedure_manager))
}

View File

@@ -33,6 +33,7 @@ use operator::statement::{StatementExecutor, StatementExecutorRef};
use operator::table::TableMutationOperator;
use partition::manager::PartitionRuleManager;
use pipeline::pipeline_operator::PipelineOperator;
use query::region_query::RegionQueryHandlerFactoryRef;
use query::stats::StatementStatistics;
use query::QueryEngineFactory;
use snafu::OptionExt;
@@ -114,7 +115,11 @@ impl FrontendBuilder {
.unwrap_or_else(|| Arc::new(DummyCacheInvalidator));
let region_query_handler =
FrontendRegionQueryHandler::arc(partition_manager.clone(), node_manager.clone());
if let Some(factory) = plugins.get::<RegionQueryHandlerFactoryRef>() {
factory.build(partition_manager.clone(), node_manager.clone())
} else {
FrontendRegionQueryHandler::arc(partition_manager.clone(), node_manager.clone())
};
let table_flownode_cache =
self.layered_cache_registry

View File

@@ -17,6 +17,7 @@ use std::sync::Arc;
use async_trait::async_trait;
use catalog::CatalogManagerRef;
use common_catalog::consts::{trace_services_table_name, TRACE_TABLE_NAME};
use common_function::function::{Function, FunctionRef};
use common_function::scalars::json::json_get::{
JsonGetBool, JsonGetFloat, JsonGetInt, JsonGetString,
@@ -28,7 +29,7 @@ use common_recordbatch::adapter::RecordBatchStreamAdapter;
use datafusion::dataframe::DataFrame;
use datafusion::execution::context::SessionContext;
use datafusion::execution::SessionStateBuilder;
use datafusion_expr::{col, lit, lit_timestamp_nano, wildcard, Expr};
use datafusion_expr::{col, lit, lit_timestamp_nano, wildcard, Expr, SortExpr};
use query::QueryEngineRef;
use serde_json::Value as JsonValue;
use servers::error::{
@@ -38,7 +39,7 @@ use servers::error::{
use servers::http::jaeger::{QueryTraceParams, JAEGER_QUERY_TABLE_NAME_KEY};
use servers::otlp::trace::{
DURATION_NANO_COLUMN, SERVICE_NAME_COLUMN, SPAN_ATTRIBUTES_COLUMN, SPAN_KIND_COLUMN,
SPAN_KIND_PREFIX, SPAN_NAME_COLUMN, TIMESTAMP_COLUMN, TRACE_ID_COLUMN, TRACE_TABLE_NAME,
SPAN_KIND_PREFIX, SPAN_NAME_COLUMN, TIMESTAMP_COLUMN, TRACE_ID_COLUMN,
};
use servers::query_handler::JaegerQueryHandler;
use session::context::QueryContextRef;
@@ -48,7 +49,7 @@ use table::table::adapter::DfTableProviderAdapter;
use super::Instance;
const DEFAULT_LIMIT: usize = 100;
const DEFAULT_LIMIT: usize = 2000;
#[async_trait]
impl JaegerQueryHandler for Instance {
@@ -60,9 +61,10 @@ impl JaegerQueryHandler for Instance {
self.query_engine(),
vec![col(SERVICE_NAME_COLUMN)],
vec![],
Some(DEFAULT_LIMIT),
vec![],
None,
true,
None,
vec![col(SERVICE_NAME_COLUMN)],
)
.await?)
}
@@ -72,6 +74,8 @@ impl JaegerQueryHandler for Instance {
ctx: QueryContextRef,
service_name: &str,
span_kind: Option<&str>,
start_time: Option<i64>,
end_time: Option<i64>,
) -> ServerResult<Output> {
let mut filters = vec![col(SERVICE_NAME_COLUMN).eq(lit(service_name))];
@@ -83,18 +87,29 @@ impl JaegerQueryHandler for Instance {
))));
}
if let Some(start_time) = start_time {
// Microseconds to nanoseconds.
filters.push(col(TIMESTAMP_COLUMN).gt_eq(lit_timestamp_nano(start_time * 1_000)));
}
if let Some(end_time) = end_time {
// Microseconds to nanoseconds.
filters.push(col(TIMESTAMP_COLUMN).lt_eq(lit_timestamp_nano(end_time * 1_000)));
}
// It's equivalent to
//
// ```
// SELECT
// span_name,
// span_kind
// SELECT DISTINCT span_name, span_kind
// FROM
// {db}.{trace_table}
// WHERE
// service_name = '{service_name}'
// service_name = '{service_name}' AND
// timestamp >= {start_time} AND
// timestamp <= {end_time} AND
// span_kind = '{span_kind}'
// ORDER BY
// timestamp
// span_name ASC
// ```.
Ok(query_trace_table(
ctx,
@@ -104,11 +119,13 @@ impl JaegerQueryHandler for Instance {
col(SPAN_NAME_COLUMN),
col(SPAN_KIND_COLUMN),
col(SERVICE_NAME_COLUMN),
col(TIMESTAMP_COLUMN),
],
filters,
vec![col(SPAN_NAME_COLUMN).sort(true, false)], // Sort by span_name in ascending order.
Some(DEFAULT_LIMIT),
None,
false,
vec![col(SPAN_NAME_COLUMN), col(SPAN_KIND_COLUMN)],
)
.await?)
}
@@ -124,7 +141,7 @@ impl JaegerQueryHandler for Instance {
// WHERE
// trace_id = '{trace_id}'
// ORDER BY
// timestamp
// timestamp DESC
// ```.
let selects = vec![wildcard()];
@@ -136,9 +153,10 @@ impl JaegerQueryHandler for Instance {
self.query_engine(),
selects,
filters,
vec![col(TIMESTAMP_COLUMN).sort(false, false)], // Sort by timestamp in descending order.
Some(DEFAULT_LIMIT),
None,
false,
vec![],
)
.await?)
}
@@ -178,9 +196,10 @@ impl JaegerQueryHandler for Instance {
self.query_engine(),
selects,
filters,
vec![col(TIMESTAMP_COLUMN).sort(false, false)], // Sort by timestamp in descending order.
Some(DEFAULT_LIMIT),
query_params.tags,
false,
vec![],
)
.await?)
}
@@ -193,14 +212,24 @@ async fn query_trace_table(
query_engine: &QueryEngineRef,
selects: Vec<Expr>,
filters: Vec<Expr>,
sorts: Vec<SortExpr>,
limit: Option<usize>,
tags: Option<HashMap<String, JsonValue>>,
distinct: bool,
distincts: Vec<Expr>,
) -> ServerResult<Output> {
let table_name = ctx
let trace_table_name = ctx
.extension(JAEGER_QUERY_TABLE_NAME_KEY)
.unwrap_or(TRACE_TABLE_NAME);
// If only select services, use the trace services table.
let table_name = {
if selects.len() == 1 && selects[0] == col(SERVICE_NAME_COLUMN) {
&trace_services_table_name(trace_table_name)
} else {
trace_table_name
}
};
let table = catalog_manager
.table(
ctx.current_catalog(),
@@ -244,13 +273,19 @@ async fn query_trace_table(
})?;
// Apply the distinct if needed.
let dataframe = if distinct {
dataframe.distinct().context(DataFusionSnafu)?
} else {
// for non distinct query, sort by timestamp to make results stable
let dataframe = if !distincts.is_empty() {
dataframe
.sort_by(vec![col(TIMESTAMP_COLUMN)])
.distinct_on(distincts.clone(), distincts, None)
.context(DataFusionSnafu)?
} else {
dataframe
};
// Apply the sorts if needed.
let dataframe = if !sorts.is_empty() {
dataframe.sort(sorts).context(DataFusionSnafu)?
} else {
dataframe
};
// Apply the limit if needed.

View File

@@ -22,9 +22,10 @@ use common_recordbatch::SendableRecordBatchStream;
use partition::manager::PartitionRuleManagerRef;
use query::error::{RegionQuerySnafu, Result as QueryResult};
use query::region_query::RegionQueryHandler;
use session::ReadPreference;
use snafu::ResultExt;
use crate::error::{FindTableRouteSnafu, RequestQuerySnafu, Result};
use crate::error::{FindRegionPeerSnafu, RequestQuerySnafu, Result};
pub(crate) struct FrontendRegionQueryHandler {
partition_manager: PartitionRuleManagerRef,
@@ -45,8 +46,12 @@ impl FrontendRegionQueryHandler {
#[async_trait]
impl RegionQueryHandler for FrontendRegionQueryHandler {
async fn do_get(&self, request: QueryRequest) -> QueryResult<SendableRecordBatchStream> {
self.do_get_inner(request)
async fn do_get(
&self,
read_preference: ReadPreference,
request: QueryRequest,
) -> QueryResult<SendableRecordBatchStream> {
self.do_get_inner(read_preference, request)
.await
.map_err(BoxedError::new)
.context(RegionQuerySnafu)
@@ -54,15 +59,20 @@ impl RegionQueryHandler for FrontendRegionQueryHandler {
}
impl FrontendRegionQueryHandler {
async fn do_get_inner(&self, request: QueryRequest) -> Result<SendableRecordBatchStream> {
async fn do_get_inner(
&self,
read_preference: ReadPreference,
request: QueryRequest,
) -> Result<SendableRecordBatchStream> {
let region_id = request.region_id;
let peer = &self
.partition_manager
.find_region_leader(region_id)
.await
.context(FindTableRouteSnafu {
table_id: region_id.table_id(),
.context(FindRegionPeerSnafu {
region_id,
read_preference,
})?;
let client = self.node_manager.datanode(peer).await;

View File

@@ -15,6 +15,7 @@
use std::collections::HashSet;
use std::ops::Range;
use fastbloom::BloomFilter;
use greptime_proto::v1::index::BloomFilterMeta;
use itertools::Itertools;
@@ -22,6 +23,14 @@ use crate::bloom_filter::error::Result;
use crate::bloom_filter::reader::BloomFilterReader;
use crate::Bytes;
/// `InListPredicate` contains a list of acceptable values. A value needs to match at least
/// one of the elements (logical OR semantic) for the predicate to be satisfied.
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct InListPredicate {
/// List of acceptable values.
pub list: HashSet<Bytes>,
}
pub struct BloomFilterApplier {
reader: Box<dyn BloomFilterReader + Send>,
meta: BloomFilterMeta,
@@ -34,66 +43,161 @@ impl BloomFilterApplier {
Ok(Self { reader, meta })
}
/// Searches ranges of rows that match the given probes in the given search range.
/// Searches ranges of rows that match all the given predicates in the search ranges.
/// Each predicate represents an OR condition of probes, and all predicates must match (AND semantics).
/// The logic is: (probe1 OR probe2 OR ...) AND (probe3 OR probe4 OR ...)
pub async fn search(
&mut self,
probes: &HashSet<Bytes>,
search_range: Range<usize>,
predicates: &[InListPredicate],
search_ranges: &[Range<usize>],
) -> Result<Vec<Range<usize>>> {
let rows_per_segment = self.meta.rows_per_segment as usize;
let start_seg = search_range.start / rows_per_segment;
let mut end_seg = search_range.end.div_ceil(rows_per_segment);
if end_seg == self.meta.segment_loc_indices.len() + 1 {
// In a previous version, there was a bug where if the last segment was all null,
// this segment would not be written into the index. This caused the slice
// `self.meta.segment_loc_indices[start_seg..end_seg]` to go out of bounds due to
// the missing segment. Since the `search` function does not search for nulls,
// we can simply ignore the last segment in this buggy scenario.
end_seg -= 1;
if predicates.is_empty() {
// If no predicates, return empty result
return Ok(Vec::new());
}
let locs = &self.meta.segment_loc_indices[start_seg..end_seg];
let segments = self.row_ranges_to_segments(search_ranges);
let (seg_locations, bloom_filters) = self.load_bloom_filters(&segments).await?;
let matching_row_ranges = self.find_matching_rows(seg_locations, bloom_filters, predicates);
Ok(intersect_ranges(search_ranges, &matching_row_ranges))
}
// dedup locs
let deduped_locs = locs
/// Converts row ranges to segment ranges and returns unique segments
fn row_ranges_to_segments(&self, row_ranges: &[Range<usize>]) -> Vec<usize> {
let rows_per_segment = self.meta.rows_per_segment as usize;
let mut segments = vec![];
for range in row_ranges {
let start_seg = range.start / rows_per_segment;
let mut end_seg = range.end.div_ceil(rows_per_segment);
if end_seg == self.meta.segment_loc_indices.len() + 1 {
// Handle legacy bug with missing last segment
//
// In a previous version, there was a bug where if the last segment was all null,
// this segment would not be written into the index. This caused the slice
// `self.meta.segment_loc_indices[start_seg..end_seg]` to go out of bounds due to
// the missing segment. Since the `search` function does not search for nulls,
// we can simply ignore the last segment in this buggy scenario.
end_seg -= 1;
}
segments.extend(start_seg..end_seg);
}
// Ensure segments are unique and sorted
segments.sort_unstable();
segments.dedup();
segments
}
/// Loads bloom filters for the given segments and returns the segment locations and bloom filters
async fn load_bloom_filters(
&mut self,
segments: &[usize],
) -> Result<(Vec<(u64, usize)>, Vec<BloomFilter>)> {
let segment_locations = segments
.iter()
.dedup()
.map(|i| self.meta.bloom_filter_locs[*i as usize])
.map(|&seg| (self.meta.segment_loc_indices[seg], seg))
.collect::<Vec<_>>();
let bfs = self.reader.bloom_filter_vec(&deduped_locs).await?;
let mut ranges: Vec<Range<usize>> = Vec::with_capacity(bfs.len());
for ((_, mut group), bloom) in locs
let bloom_filter_locs = segment_locations
.iter()
.zip(start_seg..end_seg)
.chunk_by(|(x, _)| **x)
.map(|(loc, _)| *loc)
.dedup()
.map(|i| self.meta.bloom_filter_locs[i as usize])
.collect::<Vec<_>>();
let bloom_filters = self.reader.bloom_filter_vec(&bloom_filter_locs).await?;
Ok((segment_locations, bloom_filters))
}
/// Finds segments that match all predicates and converts them to row ranges
fn find_matching_rows(
&self,
segment_locations: Vec<(u64, usize)>,
bloom_filters: Vec<BloomFilter>,
predicates: &[InListPredicate],
) -> Vec<Range<usize>> {
let rows_per_segment = self.meta.rows_per_segment as usize;
let mut matching_row_ranges = Vec::with_capacity(bloom_filters.len());
// Group segments by their location index (since they have the same bloom filter) and check if they match all predicates
for ((_loc_index, group), bloom_filter) in segment_locations
.into_iter()
.zip(bfs.iter())
.chunk_by(|(loc, _)| *loc)
.into_iter()
.zip(bloom_filters.iter())
{
let start = group.next().unwrap().1 * rows_per_segment; // SAFETY: group is not empty
let end = group.last().map_or(start + rows_per_segment, |(_, end)| {
(end + 1) * rows_per_segment
// Check if this bloom filter matches each predicate (AND semantics)
let matches_all_predicates = predicates.iter().all(|predicate| {
// For each predicate, at least one probe must match (OR semantics)
predicate
.list
.iter()
.any(|probe| bloom_filter.contains(probe))
});
let actual_start = start.max(search_range.start);
let actual_end = end.min(search_range.end);
for probe in probes {
if bloom.contains(probe) {
match ranges.last_mut() {
Some(last) if last.end == actual_start => {
last.end = actual_end;
}
_ => {
ranges.push(actual_start..actual_end);
}
}
break;
}
if !matches_all_predicates {
continue;
}
// For each matching segment, convert to row range
for (_, segment) in group {
let start_row = segment * rows_per_segment;
let end_row = (segment + 1) * rows_per_segment;
matching_row_ranges.push(start_row..end_row);
}
}
Ok(ranges)
self.merge_adjacent_ranges(matching_row_ranges)
}
/// Merges adjacent row ranges to reduce the number of ranges
fn merge_adjacent_ranges(&self, ranges: Vec<Range<usize>>) -> Vec<Range<usize>> {
ranges
.into_iter()
.coalesce(|prev, next| {
if prev.end == next.start {
Ok(prev.start..next.end)
} else {
Err((prev, next))
}
})
.collect::<Vec<_>>()
}
}
/// Intersects two lists of ranges and returns the intersection.
///
/// The input lists are assumed to be sorted and non-overlapping.
fn intersect_ranges(lhs: &[Range<usize>], rhs: &[Range<usize>]) -> Vec<Range<usize>> {
let mut i = 0;
let mut j = 0;
let mut output = Vec::new();
while i < lhs.len() && j < rhs.len() {
let r1 = &lhs[i];
let r2 = &rhs[j];
// Find intersection if exists
let start = r1.start.max(r2.start);
let end = r1.end.min(r2.end);
if start < end {
output.push(start..end);
}
// Move forward the range that ends first
if r1.end < r2.end {
i += 1;
} else {
j += 1;
}
}
output
}
#[cfg(test)]
@@ -158,37 +262,6 @@ mod tests {
vec![b"dup".to_vec()],
];
let cases = vec![
(vec![b"row00".to_vec()], 0..28, vec![0..4]), // search one row in full range
(vec![b"row05".to_vec()], 4..8, vec![4..8]), // search one row in partial range
(vec![b"row03".to_vec()], 4..8, vec![]), // search for a row that doesn't exist in the partial range
(
vec![b"row01".to_vec(), b"row06".to_vec()],
0..28,
vec![0..8],
), // search multiple rows in multiple ranges
(
vec![b"row01".to_vec(), b"row11".to_vec()],
0..28,
vec![0..4, 8..12],
), // search multiple rows in multiple ranges
(vec![b"row99".to_vec()], 0..28, vec![]), // search for a row that doesn't exist in the full range
(vec![b"row00".to_vec()], 12..12, vec![]), // search in an empty range
(
vec![b"row04".to_vec(), b"row05".to_vec()],
0..12,
vec![4..8],
), // search multiple rows in same segment
(vec![b"seg01".to_vec()], 0..28, vec![4..8]), // search rows in a segment
(vec![b"seg01".to_vec()], 6..28, vec![6..8]), // search rows in a segment in partial range
(vec![b"overl".to_vec()], 0..28, vec![0..8]), // search rows in multiple segments
(vec![b"overl".to_vec()], 2..28, vec![2..8]), // search range starts from the middle of a segment
(vec![b"overp".to_vec()], 0..10, vec![4..10]), // search range ends at the middle of a segment
(vec![b"dup".to_vec()], 0..12, vec![]), // search for a duplicate row not in the range
(vec![b"dup".to_vec()], 0..16, vec![12..16]), // search for a duplicate row in the range
(vec![b"dup".to_vec()], 0..28, vec![12..28]), // search for a duplicate row in the full range
];
for row in rows {
creator.push_row_elems(row).await.unwrap();
}
@@ -196,15 +269,215 @@ mod tests {
creator.finish(&mut writer).await.unwrap();
let bytes = writer.into_inner();
let reader = BloomFilterReaderImpl::new(bytes);
let mut applier = BloomFilterApplier::new(Box::new(reader)).await.unwrap();
for (probes, search_range, expected) in cases {
let probes: HashSet<Bytes> = probes.into_iter().collect();
let ranges = applier.search(&probes, search_range).await.unwrap();
assert_eq!(ranges, expected);
// Test cases for predicates
let cases = vec![
// Single value predicates
(
vec![InListPredicate {
list: HashSet::from_iter([b"row00".to_vec()]),
}],
0..28,
vec![0..4],
),
(
vec![InListPredicate {
list: HashSet::from_iter([b"row05".to_vec()]),
}],
4..8,
vec![4..8],
),
(
vec![InListPredicate {
list: HashSet::from_iter([b"row03".to_vec()]),
}],
4..8,
vec![],
),
// Multiple values in a single predicate (OR logic)
(
vec![InListPredicate {
list: HashSet::from_iter([b"overl".to_vec(), b"row06".to_vec()]),
}],
0..28,
vec![0..8],
),
(
vec![InListPredicate {
list: HashSet::from_iter([b"seg01".to_vec(), b"overp".to_vec()]),
}],
0..28,
vec![4..12],
),
// Non-existent values
(
vec![InListPredicate {
list: HashSet::from_iter([b"row99".to_vec()]),
}],
0..28,
vec![],
),
// Empty range
(
vec![InListPredicate {
list: HashSet::from_iter([b"row00".to_vec()]),
}],
12..12,
vec![],
),
// Multiple values in a single predicate within specific ranges
(
vec![InListPredicate {
list: HashSet::from_iter([b"row04".to_vec(), b"row05".to_vec()]),
}],
0..12,
vec![4..8],
),
(
vec![InListPredicate {
list: HashSet::from_iter([b"seg01".to_vec()]),
}],
0..28,
vec![4..8],
),
(
vec![InListPredicate {
list: HashSet::from_iter([b"seg01".to_vec()]),
}],
6..28,
vec![6..8],
),
// Values spanning multiple segments
(
vec![InListPredicate {
list: HashSet::from_iter([b"overl".to_vec()]),
}],
0..28,
vec![0..8],
),
(
vec![InListPredicate {
list: HashSet::from_iter([b"overl".to_vec()]),
}],
2..28,
vec![2..8],
),
(
vec![InListPredicate {
list: HashSet::from_iter([b"overp".to_vec()]),
}],
0..10,
vec![4..10],
),
// Duplicate values
(
vec![InListPredicate {
list: HashSet::from_iter([b"dup".to_vec()]),
}],
0..12,
vec![],
),
(
vec![InListPredicate {
list: HashSet::from_iter([b"dup".to_vec()]),
}],
0..16,
vec![12..16],
),
(
vec![InListPredicate {
list: HashSet::from_iter([b"dup".to_vec()]),
}],
0..28,
vec![12..28],
),
// Multiple predicates (AND logic)
(
vec![
InListPredicate {
list: HashSet::from_iter([b"row00".to_vec(), b"row01".to_vec()]),
},
InListPredicate {
list: HashSet::from_iter([b"seg00".to_vec()]),
},
],
0..28,
vec![0..4],
),
(
vec![
InListPredicate {
list: HashSet::from_iter([b"overl".to_vec()]),
},
InListPredicate {
list: HashSet::from_iter([b"seg01".to_vec()]),
},
],
0..28,
vec![4..8],
),
];
for (predicates, search_range, expected) in cases {
let result = applier.search(&predicates, &[search_range]).await.unwrap();
assert_eq!(
result, expected,
"Expected {:?}, got {:?}",
expected, result
);
}
}
#[test]
#[allow(clippy::single_range_in_vec_init)]
fn test_intersect_ranges() {
// empty inputs
assert_eq!(intersect_ranges(&[], &[]), Vec::<Range<usize>>::new());
assert_eq!(intersect_ranges(&[1..5], &[]), Vec::<Range<usize>>::new());
assert_eq!(intersect_ranges(&[], &[1..5]), Vec::<Range<usize>>::new());
// no overlap
assert_eq!(
intersect_ranges(&[1..3, 5..7], &[3..5, 7..9]),
Vec::<Range<usize>>::new()
);
// single overlap
assert_eq!(intersect_ranges(&[1..5], &[3..7]), vec![3..5]);
// multiple overlaps
assert_eq!(
intersect_ranges(&[1..5, 7..10, 12..15], &[2..6, 8..13]),
vec![2..5, 8..10, 12..13]
);
// exact overlap
assert_eq!(
intersect_ranges(&[1..3, 5..7], &[1..3, 5..7]),
vec![1..3, 5..7]
);
// contained ranges
assert_eq!(
intersect_ranges(&[1..10], &[2..4, 5..7, 8..9]),
vec![2..4, 5..7, 8..9]
);
// partial overlaps
assert_eq!(
intersect_ranges(&[1..4, 6..9], &[2..7, 8..10]),
vec![2..4, 6..7, 8..9]
);
// single point overlap
assert_eq!(
intersect_ranges(&[1..3], &[3..5]),
Vec::<Range<usize>>::new()
);
// large ranges
assert_eq!(intersect_ranges(&[0..100], &[50..150]), vec![50..100]);
}
}

View File

@@ -12,6 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::collections::HashMap;
use std::sync::atomic::AtomicUsize;
use std::sync::Arc;
@@ -26,16 +27,20 @@ use crate::external_provider::ExternalTempFileProvider;
use crate::fulltext_index::create::FulltextIndexCreator;
use crate::fulltext_index::error::{
AbortedSnafu, BiErrorsSnafu, BloomFilterFinishSnafu, ExternalSnafu, PuffinAddBlobSnafu, Result,
SerializeToJsonSnafu,
};
use crate::fulltext_index::tokenizer::{Analyzer, ChineseTokenizer, EnglishTokenizer};
use crate::fulltext_index::Config;
const PIPE_BUFFER_SIZE_FOR_SENDING_BLOB: usize = 8192;
pub const KEY_FULLTEXT_CONFIG: &str = "fulltext_config";
/// `BloomFilterFulltextIndexCreator` is for creating a fulltext index using a bloom filter.
pub struct BloomFilterFulltextIndexCreator {
inner: Option<BloomFilterCreator>,
analyzer: Analyzer,
config: Config,
}
impl BloomFilterFulltextIndexCreator {
@@ -61,6 +66,7 @@ impl BloomFilterFulltextIndexCreator {
Self {
inner: Some(inner),
analyzer,
config,
}
}
}
@@ -89,10 +95,17 @@ impl FulltextIndexCreator for BloomFilterFulltextIndexCreator {
let (tx, rx) = tokio::io::duplex(PIPE_BUFFER_SIZE_FOR_SENDING_BLOB);
let property_key = KEY_FULLTEXT_CONFIG.to_string();
let property_value = serde_json::to_string(&self.config).context(SerializeToJsonSnafu)?;
let (index_finish, puffin_add_blob) = futures::join!(
creator.finish(tx.compat_write()),
// TODO(zhongzc): add fulltext config properties
puffin_writer.put_blob(blob_key, rx.compat(), put_options, Default::default())
puffin_writer.put_blob(
blob_key,
rx.compat(),
put_options,
HashMap::from([(property_key, property_value)]),
)
);
match (

View File

@@ -104,6 +104,22 @@ pub enum Error {
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Failed to serialize to json"))]
SerializeToJson {
#[snafu(source)]
error: serde_json::error::Error,
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Failed to deserialize from json"))]
DeserializeFromJson {
#[snafu(source)]
error: serde_json::error::Error,
#[snafu(implicit)]
location: Location,
},
}
impl ErrorExt for Error {
@@ -122,6 +138,8 @@ impl ErrorExt for Error {
PuffinAddBlob { source, .. } => source.status_code(),
External { source, .. } => source.status_code(),
SerializeToJson { .. } | DeserializeFromJson { .. } => StatusCode::Internal,
}
}

View File

@@ -15,6 +15,7 @@
use std::any::Any;
use common_error::ext::ErrorExt;
use common_error::status_code::StatusCode;
use common_macro::stack_trace_debug;
use common_runtime::error::Error as RuntimeError;
use serde_json::error::Error as JsonError;
@@ -310,10 +311,72 @@ pub enum Error {
},
}
pub type Result<T> = std::result::Result<T, Error>;
fn rskafka_client_error_to_status_code(error: &rskafka::client::error::Error) -> StatusCode {
match error {
rskafka::client::error::Error::Connection(_)
| rskafka::client::error::Error::Request(_)
| rskafka::client::error::Error::InvalidResponse(_)
| rskafka::client::error::Error::ServerError { .. }
| rskafka::client::error::Error::RetryFailed(_) => StatusCode::Internal,
rskafka::client::error::Error::Timeout => StatusCode::StorageUnavailable,
_ => StatusCode::Internal,
}
}
impl ErrorExt for Error {
fn as_any(&self) -> &dyn Any {
self
}
}
pub type Result<T> = std::result::Result<T, Error>;
fn status_code(&self) -> StatusCode {
use Error::*;
match self {
TlsConfig { .. }
| InvalidProvider { .. }
| IllegalNamespace { .. }
| MissingKey { .. }
| MissingValue { .. }
| OverrideCompactedEntry { .. } => StatusCode::InvalidArguments,
StartWalTask { .. }
| StopWalTask { .. }
| IllegalState { .. }
| ResolveKafkaEndpoint { .. }
| NoMaxValue { .. }
| Cast { .. }
| EncodeJson { .. }
| DecodeJson { .. }
| IllegalSequence { .. }
| DiscontinuousLogIndex { .. }
| OrderedBatchProducerStopped { .. }
| WaitProduceResultReceiver { .. }
| WaitDumpIndex { .. }
| MetaLengthExceededLimit { .. } => StatusCode::Internal,
// Object store related errors
CreateWriter { .. } | WriteIndex { .. } | ReadIndex { .. } | Io { .. } => {
StatusCode::StorageUnavailable
}
// Raft engine
FetchEntry { .. } | RaftEngine { .. } | AddEntryLogBatch { .. } => {
StatusCode::StorageUnavailable
}
// Kafka producer related errors
ProduceRecord { error, .. } => match error {
rskafka::client::producer::Error::Client(error) => {
rskafka_client_error_to_status_code(error)
}
rskafka::client::producer::Error::Aggregator(_)
| rskafka::client::producer::Error::FlushError(_)
| rskafka::client::producer::Error::TooLarge => StatusCode::Internal,
},
BuildClient { error, .. }
| BuildPartitionClient { error, .. }
| BatchProduce { error, .. }
| GetOffset { error, .. }
| ConsumeRecord { error, .. } => rskafka_client_error_to_status_code(error),
}
}
}

View File

@@ -20,6 +20,7 @@ pub(crate) mod producer;
pub(crate) mod util;
pub(crate) mod worker;
pub use client_manager::DEFAULT_PARTITION;
pub use index::{default_index_file, GlobalIndexCollector};
use serde::{Deserialize, Serialize};
use store_api::logstore::entry::Id as EntryId;

View File

@@ -15,10 +15,10 @@
use std::collections::HashMap;
use std::sync::Arc;
use common_wal::config::kafka::common::DEFAULT_BACKOFF_CONFIG;
use common_wal::config::kafka::DatanodeKafkaConfig;
use rskafka::client::partition::{Compression, PartitionClient, UnknownTopicHandling};
use rskafka::client::ClientBuilder;
use rskafka::BackoffConfig;
use snafu::ResultExt;
use store_api::logstore::provider::KafkaProvider;
use tokio::sync::{Mutex, RwLock};
@@ -31,7 +31,7 @@ use crate::kafka::producer::{OrderedBatchProducer, OrderedBatchProducerRef};
// Each topic only has one partition for now.
// The `DEFAULT_PARTITION` refers to the index of the partition.
const DEFAULT_PARTITION: i32 = 0;
pub const DEFAULT_PARTITION: i32 = 0;
/// Arc wrapper of ClientManager.
pub(crate) type ClientManagerRef = Arc<ClientManager>;
@@ -73,16 +73,11 @@ impl ClientManager {
global_index_collector: Option<GlobalIndexCollector>,
) -> Result<Self> {
// Sets backoff config for the top-level kafka client and all clients constructed by it.
let backoff_config = BackoffConfig {
init_backoff: config.backoff.init,
max_backoff: config.backoff.max,
base: config.backoff.base as f64,
deadline: config.backoff.deadline,
};
let broker_endpoints = common_wal::resolve_to_ipv4(&config.connection.broker_endpoints)
.await
.context(ResolveKafkaEndpointSnafu)?;
let mut builder = ClientBuilder::new(broker_endpoints).backoff_config(backoff_config);
let mut builder =
ClientBuilder::new(broker_endpoints).backoff_config(DEFAULT_BACKOFF_CONFIG);
if let Some(sasl) = &config.connection.sasl {
builder = builder.sasl_config(sasl.config.clone().into_sasl_config());
};

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