Compare commits

...

19 Commits

Author SHA1 Message Date
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
150 changed files with 5291 additions and 790 deletions

106
Cargo.lock generated
View File

@@ -1053,7 +1053,7 @@ dependencies = [
"bitflags 2.9.0",
"cexpr",
"clang-sys",
"itertools 0.13.0",
"itertools 0.11.0",
"proc-macro2",
"quote",
"regex",
@@ -2038,6 +2038,7 @@ dependencies = [
"h3o",
"hyperloglogplus",
"jsonb",
"memchr",
"nalgebra 0.33.2",
"num",
"num-traits",
@@ -2344,6 +2345,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"
@@ -4091,6 +4099,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 +4643,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 +4690,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=fb8e20ce29afd81835e3ea3c1164c8ce10de2c65#fb8e20ce29afd81835e3ea3c1164c8ce10de2c65"
dependencies = [
"prost 0.13.3",
"serde",
@@ -4750,6 +4769,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"
@@ -5815,10 +5844,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",
]
@@ -6223,7 +6253,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "4979f22fdb869068da03c9f7528f8297c6fd2606bc3a4affe42e6a823fdb8da4"
dependencies = [
"cfg-if",
"windows-targets 0.52.6",
"windows-targets 0.48.5",
]
[[package]]
@@ -6691,12 +6721,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",
@@ -8083,6 +8115,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)",
@@ -8385,6 +8418,7 @@ dependencies = [
"serde",
"serde_json",
"session",
"simd-json",
"snafu 0.8.5",
"sql",
"table",
@@ -8854,7 +8888,7 @@ checksum = "0c1318b19085f08681016926435853bbf7858f9c082d0999b80550ff5d9abe15"
dependencies = [
"bytes",
"heck 0.5.0",
"itertools 0.13.0",
"itertools 0.11.0",
"log",
"multimap",
"once_cell",
@@ -8900,7 +8934,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "e9552f850d5f0964a4e4d0bf306459ac29323ddfbae05e35a7c0d35cb0803cc5"
dependencies = [
"anyhow",
"itertools 0.13.0",
"itertools 0.11.0",
"proc-macro2",
"quote",
"syn 2.0.96",
@@ -9094,6 +9128,7 @@ dependencies = [
"num-traits",
"object-store",
"once_cell",
"partition",
"paste",
"pretty_assertions",
"prometheus",
@@ -10532,6 +10567,7 @@ dependencies = [
"common-query",
"common-recordbatch",
"common-runtime",
"common-session",
"common-telemetry",
"common-test-util",
"common-time",
@@ -10592,6 +10628,7 @@ dependencies = [
"serde",
"serde_json",
"session",
"simd-json",
"snafu 0.8.5",
"snap",
"socket2",
@@ -10627,6 +10664,7 @@ dependencies = [
"common-error",
"common-macro",
"common-recordbatch",
"common-session",
"common-telemetry",
"common-time",
"derive_builder 0.20.1",
@@ -10730,6 +10768,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"
@@ -13194,6 +13247,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 +13349,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 +13387,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 +13397,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 +13410,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"
@@ -13489,7 +13557,7 @@ version = "0.1.9"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "cf221c93e13a30d793f7645a0e7762c55d169dbb0a49671918a2319d289b10bb"
dependencies = [
"windows-sys 0.59.0",
"windows-sys 0.48.0",
]
[[package]]

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 = "fb8e20ce29afd81835e3ea3c1164c8ce10de2c65" }
hex = "0.4"
http = "1"
humantime = "2.1"
@@ -185,6 +186,7 @@ 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"
@@ -247,6 +249,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" }

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. |
@@ -98,6 +97,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 +328,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. |
@@ -381,7 +382,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. |
@@ -551,7 +551,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

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]

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"
@@ -302,6 +299,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

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

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

@@ -850,6 +850,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;
@@ -893,6 +894,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

@@ -512,6 +512,10 @@ impl TableMetadataManager {
&self.table_route_manager
}
pub fn topic_region_manager(&self) -> &TopicRegionManager {
&self.topic_region_manager
}
#[cfg(feature = "testing")]
pub fn kv_backend(&self) -> &KvBackendRef {
&self.kv_backend

View File

@@ -39,6 +39,7 @@ pub mod node_manager;
pub mod peer;
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,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

@@ -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,6 +12,8 @@
// 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::MetasrvKafkaConfig;
use rskafka::client::error::Error as RsKafkaError;
@@ -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()
@@ -151,7 +159,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

@@ -58,6 +58,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,
@@ -192,7 +199,8 @@ impl ErrorExt for Error {
| Error::FromJson { .. }
| Error::WaitWatcher { .. }
| Error::RetryLater { .. }
| Error::RollbackProcedureRecovered { .. } => StatusCode::Internal,
| Error::RollbackProcedureRecovered { .. }
| Error::TooManyRunningProcedures { .. } => StatusCode::Internal,
Error::RetryTimesExceeded { .. }
| Error::RollbackTimesExceeded { .. }

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};
@@ -33,6 +34,7 @@ use self::rwlock::KeyRwLock;
use crate::error::{
self, DuplicateProcedureSnafu, Error, LoaderConflictSnafu, ManagerNotStartSnafu, Result,
StartRemoveOutdatedMetaTaskSnafu, StopRemoveOutdatedMetaTaskSnafu,
TooManyRunningProceduresSnafu,
};
use crate::local::runner::Runner;
use crate::procedure::{BoxedProcedureLoader, InitProcedureState, ProcedureInfo};
@@ -147,7 +149,6 @@ type ProcedureMetaRef = Arc<ProcedureMeta>;
/// Procedure loaded from store.
struct LoadedProcedure {
procedure: BoxedProcedure,
parent_id: Option<ProcedureId>,
step: u32,
}
@@ -157,8 +158,7 @@ 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.
@@ -179,7 +179,7 @@ impl 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)),
}
@@ -210,18 +210,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 +273,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 +300,6 @@ impl ManagerContext {
Some(LoadedProcedure {
procedure,
parent_id: message.parent_id,
step: message.step,
})
}
@@ -350,23 +348,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 +404,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 +415,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,
}
}
}
@@ -492,6 +488,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),
@@ -1119,6 +1122,7 @@ 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);
@@ -1191,6 +1195,69 @@ 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 manager = LocalManager::new(config, state_store);
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,

View File

@@ -348,24 +348,14 @@ 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,

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

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

@@ -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_ext_handler: 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_ext_handler: 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_ext_handler,
}
}
@@ -85,6 +92,7 @@ impl RegionAliveKeeper {
let handle = Arc::new(CountdownTaskHandle::new(
self.region_server.clone(),
self.countdown_task_ext_handler.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,27 @@ 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 CountdownTaskExtHandler>;
/// Extension trait for [CountdownTaskHandle] to reset deadline method.
#[async_trait]
pub trait CountdownTaskExtHandler: Send + Sync {
async fn reset_deadline(
&self,
region_server: &RegionServer,
role: RegionRole,
deadline: Instant,
extension_info: HashMap<String, Vec<u8>>,
);
}
struct CountdownTaskHandle {
tx: mpsc::Sender<CountdownCommand>,
handler: JoinHandle<()>,
@@ -280,11 +304,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 +352,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 +384,7 @@ impl Drop for CountdownTaskHandle {
struct CountdownTask {
region_server: RegionServer,
region_id: RegionId,
handler_ext: Option<CountdownTaskHandlerExtRef>,
rx: mpsc::Receiver<CountdownCommand>,
}
@@ -379,8 +414,18 @@ 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,
role,
deadline,
extension_info,
).await;
}
trace!(
"Reset deadline of region {region_id} to approximately {} seconds later.",
(deadline - Instant::now()).as_secs_f32(),
@@ -402,7 +447,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 +478,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 +515,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 +540,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 +571,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

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

@@ -118,6 +118,7 @@ impl Instance {
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));

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

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

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

View File

@@ -8,6 +8,7 @@ license.workspace = true
mock = []
pg_kvbackend = ["dep:tokio-postgres", "common-meta/pg_kvbackend", "dep:deadpool-postgres", "dep:deadpool"]
mysql_kvbackend = ["dep:sqlx", "common-meta/mysql_kvbackend"]
testing = ["common-wal/testing"]
[lints]
workspace = true
@@ -51,12 +52,14 @@ humantime-serde.workspace = true
hyper-util = { workspace = true, features = ["tokio"] }
itertools.workspace = true
lazy_static.workspace = true
log-store.workspace = true
once_cell.workspace = true
parking_lot.workspace = true
prometheus.workspace = true
prost.workspace = true
rand.workspace = true
regex.workspace = true
rskafka.workspace = true
serde.workspace = true
serde_json.workspace = true
servers.workspace = true

View File

@@ -787,6 +787,31 @@ pub enum Error {
location: Location,
source: common_meta::error::Error,
},
#[snafu(display(
"Failed to build a Kafka partition client, topic: {}, partition: {}",
topic,
partition
))]
BuildPartitionClient {
topic: String,
partition: i32,
#[snafu(implicit)]
location: Location,
#[snafu(source)]
error: rskafka::client::error::Error,
},
#[snafu(display("Failed to delete record from Kafka"))]
DeleteRecord {
#[snafu(implicit)]
location: Location,
#[snafu(source)]
error: rskafka::client::error::Error,
topic: String,
partition: i32,
offset: u64,
},
}
impl Error {
@@ -834,7 +859,9 @@ impl ErrorExt for Error {
| Error::ExceededDeadline { .. }
| Error::ChooseItems { .. }
| Error::FlowStateHandler { .. }
| Error::BuildWalOptionsAllocator { .. } => StatusCode::Internal,
| Error::BuildWalOptionsAllocator { .. }
| Error::BuildPartitionClient { .. }
| Error::DeleteRecord { .. } => StatusCode::Internal,
Error::Unsupported { .. } => StatusCode::Unsupported,

View File

@@ -28,6 +28,7 @@ use collect_cluster_info_handler::{
CollectDatanodeClusterInfoHandler, CollectFlownodeClusterInfoHandler,
CollectFrontendClusterInfoHandler,
};
use collect_leader_region_handler::CollectLeaderRegionHandler;
use collect_stats_handler::CollectStatsHandler;
use common_base::Plugins;
use common_meta::datanode::Stat;
@@ -62,6 +63,7 @@ use crate::service::mailbox::{
pub mod check_leader_handler;
pub mod collect_cluster_info_handler;
pub mod collect_leader_region_handler;
pub mod collect_stats_handler;
pub mod extract_stat_handler;
pub mod failure_handler;
@@ -96,7 +98,7 @@ pub trait HeartbeatHandler: Send + Sync {
/// HandleControl
///
/// Controls process of handling heartbeat request.
#[derive(PartialEq)]
#[derive(PartialEq, Debug)]
pub enum HandleControl {
Continue,
Done,
@@ -331,7 +333,7 @@ pub struct HeartbeatMailbox {
}
impl HeartbeatMailbox {
pub(crate) fn json_reply(msg: &MailboxMessage) -> Result<InstructionReply> {
pub fn json_reply(msg: &MailboxMessage) -> Result<InstructionReply> {
let Payload::Json(payload) =
msg.payload
.as_ref()
@@ -570,6 +572,7 @@ impl HeartbeatHandlerGroupBuilder {
if let Some(publish_heartbeat_handler) = publish_heartbeat_handler {
self.add_handler_last(publish_heartbeat_handler);
}
self.add_handler_last(CollectLeaderRegionHandler);
self.add_handler_last(CollectStatsHandler::new(self.flush_stats_factor));
self.add_handler_last(RemapFlowPeerHandler::default());
@@ -848,7 +851,7 @@ mod tests {
.unwrap();
let handlers = group.handlers;
assert_eq!(13, handlers.len());
assert_eq!(14, handlers.len());
let names = [
"ResponseHeaderHandler",
@@ -862,6 +865,7 @@ mod tests {
"CollectFlownodeClusterInfoHandler",
"MailboxHandler",
"FilterInactiveRegionStatsHandler",
"CollectLeaderRegionHandler",
"CollectStatsHandler",
"RemapFlowPeerHandler",
];
@@ -884,7 +888,7 @@ mod tests {
let group = builder.build().unwrap();
let handlers = group.handlers;
assert_eq!(14, handlers.len());
assert_eq!(15, handlers.len());
let names = [
"ResponseHeaderHandler",
@@ -899,6 +903,7 @@ mod tests {
"MailboxHandler",
"CollectStatsHandler",
"FilterInactiveRegionStatsHandler",
"CollectLeaderRegionHandler",
"CollectStatsHandler",
"RemapFlowPeerHandler",
];
@@ -918,7 +923,7 @@ mod tests {
let group = builder.build().unwrap();
let handlers = group.handlers;
assert_eq!(14, handlers.len());
assert_eq!(15, handlers.len());
let names = [
"CollectStatsHandler",
@@ -933,6 +938,7 @@ mod tests {
"CollectFlownodeClusterInfoHandler",
"MailboxHandler",
"FilterInactiveRegionStatsHandler",
"CollectLeaderRegionHandler",
"CollectStatsHandler",
"RemapFlowPeerHandler",
];
@@ -952,7 +958,7 @@ mod tests {
let group = builder.build().unwrap();
let handlers = group.handlers;
assert_eq!(14, handlers.len());
assert_eq!(15, handlers.len());
let names = [
"ResponseHeaderHandler",
@@ -967,6 +973,7 @@ mod tests {
"MailboxHandler",
"CollectStatsHandler",
"FilterInactiveRegionStatsHandler",
"CollectLeaderRegionHandler",
"CollectStatsHandler",
"RemapFlowPeerHandler",
];
@@ -986,7 +993,7 @@ mod tests {
let group = builder.build().unwrap();
let handlers = group.handlers;
assert_eq!(14, handlers.len());
assert_eq!(15, handlers.len());
let names = [
"ResponseHeaderHandler",
@@ -1000,6 +1007,7 @@ mod tests {
"CollectFlownodeClusterInfoHandler",
"MailboxHandler",
"FilterInactiveRegionStatsHandler",
"CollectLeaderRegionHandler",
"CollectStatsHandler",
"ResponseHeaderHandler",
"RemapFlowPeerHandler",
@@ -1020,7 +1028,7 @@ mod tests {
let group = builder.build().unwrap();
let handlers = group.handlers;
assert_eq!(13, handlers.len());
assert_eq!(14, handlers.len());
let names = [
"ResponseHeaderHandler",
@@ -1034,6 +1042,7 @@ mod tests {
"CollectFlownodeClusterInfoHandler",
"CollectStatsHandler",
"FilterInactiveRegionStatsHandler",
"CollectLeaderRegionHandler",
"CollectStatsHandler",
"RemapFlowPeerHandler",
];
@@ -1053,7 +1062,7 @@ mod tests {
let group = builder.build().unwrap();
let handlers = group.handlers;
assert_eq!(13, handlers.len());
assert_eq!(14, handlers.len());
let names = [
"ResponseHeaderHandler",
@@ -1067,6 +1076,7 @@ mod tests {
"CollectFlownodeClusterInfoHandler",
"MailboxHandler",
"FilterInactiveRegionStatsHandler",
"CollectLeaderRegionHandler",
"ResponseHeaderHandler",
"RemapFlowPeerHandler",
];
@@ -1086,7 +1096,7 @@ mod tests {
let group = builder.build().unwrap();
let handlers = group.handlers;
assert_eq!(13, handlers.len());
assert_eq!(14, handlers.len());
let names = [
"CollectStatsHandler",
@@ -1100,6 +1110,7 @@ mod tests {
"CollectFlownodeClusterInfoHandler",
"MailboxHandler",
"FilterInactiveRegionStatsHandler",
"CollectLeaderRegionHandler",
"CollectStatsHandler",
"RemapFlowPeerHandler",
];

View File

@@ -0,0 +1,233 @@
// 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 api::v1::meta::{HeartbeatRequest, Role};
use common_meta::region_registry::LeaderRegion;
use common_telemetry::info;
use store_api::region_engine::RegionRole;
use crate::error::Result;
use crate::handler::{HandleControl, HeartbeatAccumulator, HeartbeatHandler};
use crate::metasrv::Context;
pub struct CollectLeaderRegionHandler;
#[async_trait::async_trait]
impl HeartbeatHandler for CollectLeaderRegionHandler {
fn is_acceptable(&self, role: Role) -> bool {
role == Role::Datanode
}
async fn handle(
&self,
_req: &HeartbeatRequest,
ctx: &mut Context,
acc: &mut HeartbeatAccumulator,
) -> Result<HandleControl> {
let Some(current_stat) = acc.stat.as_ref() else {
return Ok(HandleControl::Continue);
};
let mut key_values = Vec::with_capacity(current_stat.region_stats.len());
for stat in current_stat.region_stats.iter() {
if stat.role != RegionRole::Leader {
continue;
}
let manifest = stat.region_manifest.into();
let value = LeaderRegion {
datanode_id: current_stat.id,
manifest,
};
key_values.push((stat.id, value));
}
info!("collect leader region: {:?}", key_values);
ctx.leader_region_registry.batch_put(key_values);
Ok(HandleControl::Continue)
}
}
#[cfg(test)]
mod tests {
use std::sync::Arc;
use common_meta::cache_invalidator::DummyCacheInvalidator;
use common_meta::datanode::{RegionManifestInfo, RegionStat, Stat};
use common_meta::key::TableMetadataManager;
use common_meta::kv_backend::memory::MemoryKvBackend;
use common_meta::region_registry::{LeaderRegionManifestInfo, LeaderRegionRegistry};
use common_meta::sequence::SequenceBuilder;
use store_api::region_engine::RegionRole;
use store_api::storage::RegionId;
use super::*;
use crate::cluster::MetaPeerClientBuilder;
use crate::handler::{HeartbeatMailbox, Pushers};
use crate::service::store::cached_kv::LeaderCachedKvBackend;
fn mock_ctx() -> Context {
let in_memory = Arc::new(MemoryKvBackend::new());
let kv_backend = Arc::new(MemoryKvBackend::new());
let leader_cached_kv_backend = Arc::new(LeaderCachedKvBackend::with_always_leader(
kv_backend.clone(),
));
let seq = SequenceBuilder::new("test_seq", kv_backend.clone()).build();
let mailbox = HeartbeatMailbox::create(Pushers::default(), seq);
let meta_peer_client = MetaPeerClientBuilder::default()
.election(None)
.in_memory(in_memory.clone())
.build()
.map(Arc::new)
// Safety: all required fields set at initialization
.unwrap();
Context {
server_addr: "127.0.0.1:0000".to_string(),
in_memory,
kv_backend: kv_backend.clone(),
leader_cached_kv_backend,
meta_peer_client,
mailbox,
election: None,
is_infancy: false,
table_metadata_manager: Arc::new(TableMetadataManager::new(kv_backend.clone())),
cache_invalidator: Arc::new(DummyCacheInvalidator),
leader_region_registry: Arc::new(LeaderRegionRegistry::new()),
}
}
fn new_region_stat(id: RegionId, manifest_version: u64, role: RegionRole) -> RegionStat {
RegionStat {
id,
region_manifest: RegionManifestInfo::Mito {
manifest_version,
flushed_entry_id: 0,
},
rcus: 0,
wcus: 0,
approximate_bytes: 0,
engine: "mito".to_string(),
role,
num_rows: 0,
memtable_size: 0,
manifest_size: 0,
sst_size: 0,
index_size: 0,
}
}
#[tokio::test]
async fn test_handle_collect_leader_region() {
let mut ctx = mock_ctx();
let mut acc = HeartbeatAccumulator {
stat: Some(Stat {
id: 1,
region_stats: vec![
new_region_stat(RegionId::new(1, 1), 1, RegionRole::Leader),
new_region_stat(RegionId::new(1, 2), 2, RegionRole::Follower),
],
addr: "127.0.0.1:0000".to_string(),
region_num: 2,
..Default::default()
}),
..Default::default()
};
let handler = CollectLeaderRegionHandler;
let control = handler
.handle(&HeartbeatRequest::default(), &mut ctx, &mut acc)
.await
.unwrap();
assert_eq!(control, HandleControl::Continue);
let regions = ctx
.leader_region_registry
.batch_get(vec![RegionId::new(1, 1), RegionId::new(1, 2)].into_iter());
assert_eq!(regions.len(), 1);
assert_eq!(
regions.get(&RegionId::new(1, 1)),
Some(&LeaderRegion {
datanode_id: 1,
manifest: LeaderRegionManifestInfo::Mito {
manifest_version: 1,
flushed_entry_id: 0,
},
})
);
// New heartbeat with new manifest version
acc.stat = Some(Stat {
id: 1,
region_stats: vec![new_region_stat(RegionId::new(1, 1), 2, RegionRole::Leader)],
timestamp_millis: 0,
addr: "127.0.0.1:0000".to_string(),
region_num: 1,
node_epoch: 0,
..Default::default()
});
let control = handler
.handle(&HeartbeatRequest::default(), &mut ctx, &mut acc)
.await
.unwrap();
assert_eq!(control, HandleControl::Continue);
let regions = ctx
.leader_region_registry
.batch_get(vec![RegionId::new(1, 1)].into_iter());
assert_eq!(regions.len(), 1);
assert_eq!(
regions.get(&RegionId::new(1, 1)),
Some(&LeaderRegion {
datanode_id: 1,
manifest: LeaderRegionManifestInfo::Mito {
manifest_version: 2,
flushed_entry_id: 0,
},
})
);
// New heartbeat with old manifest version
acc.stat = Some(Stat {
id: 1,
region_stats: vec![new_region_stat(RegionId::new(1, 1), 1, RegionRole::Leader)],
timestamp_millis: 0,
addr: "127.0.0.1:0000".to_string(),
region_num: 1,
node_epoch: 0,
..Default::default()
});
let control = handler
.handle(&HeartbeatRequest::default(), &mut ctx, &mut acc)
.await
.unwrap();
assert_eq!(control, HandleControl::Continue);
let regions = ctx
.leader_region_registry
.batch_get(vec![RegionId::new(1, 1)].into_iter());
assert_eq!(regions.len(), 1);
assert_eq!(
regions.get(&RegionId::new(1, 1)),
// The manifest version is not updated
Some(&LeaderRegion {
datanode_id: 1,
manifest: LeaderRegionManifestInfo::Mito {
manifest_version: 2,
flushed_entry_id: 0,
},
})
);
}
}

View File

@@ -223,6 +223,7 @@ mod tests {
use common_meta::datanode::DatanodeStatKey;
use common_meta::key::TableMetadataManager;
use common_meta::kv_backend::memory::MemoryKvBackend;
use common_meta::region_registry::LeaderRegionRegistry;
use common_meta::sequence::SequenceBuilder;
use super::*;
@@ -257,6 +258,7 @@ mod tests {
is_infancy: false,
table_metadata_manager: Arc::new(TableMetadataManager::new(kv_backend.clone())),
cache_invalidator: Arc::new(DummyCacheInvalidator),
leader_region_registry: Arc::new(LeaderRegionRegistry::new()),
};
let handler = CollectStatsHandler::default();

View File

@@ -64,7 +64,7 @@ impl HeartbeatHandler for RegionFailureHandler {
mod tests {
use api::v1::meta::HeartbeatRequest;
use common_catalog::consts::default_engine;
use common_meta::datanode::{RegionStat, Stat};
use common_meta::datanode::{RegionManifestInfo, RegionStat, Stat};
use store_api::region_engine::RegionRole;
use store_api::storage::RegionId;
use tokio::sync::oneshot;
@@ -98,6 +98,10 @@ mod tests {
manifest_size: 0,
sst_size: 0,
index_size: 0,
region_manifest: RegionManifestInfo::Mito {
manifest_version: 0,
flushed_entry_id: 0,
},
}
}
acc.stat = Some(Stat {

View File

@@ -12,13 +12,15 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::collections::HashMap;
use std::sync::Arc;
use api::v1::meta::{HeartbeatRequest, RegionLease, Role};
use async_trait::async_trait;
use common_meta::key::TableMetadataManagerRef;
use common_meta::region_keeper::MemoryRegionKeeperRef;
use store_api::region_engine::GrantedRegion;
use store_api::region_engine::{GrantedRegion, RegionRole};
use store_api::storage::RegionId;
use crate::error::Result;
use crate::handler::{HandleControl, HeartbeatAccumulator, HeartbeatHandler};
@@ -29,6 +31,17 @@ use crate::region::RegionLeaseKeeper;
pub struct RegionLeaseHandler {
region_lease_seconds: u64,
region_lease_keeper: RegionLeaseKeeperRef,
customized_region_lease_renewer: Option<CustomizedRegionLeaseRenewerRef>,
}
pub type CustomizedRegionLeaseRenewerRef = Arc<dyn CustomizedRegionLeaseRenewer>;
pub trait CustomizedRegionLeaseRenewer: Send + Sync {
fn renew(
&self,
ctx: &mut Context,
regions: HashMap<RegionId, RegionRole>,
) -> Vec<GrantedRegion>;
}
impl RegionLeaseHandler {
@@ -36,6 +49,7 @@ impl RegionLeaseHandler {
region_lease_seconds: u64,
table_metadata_manager: TableMetadataManagerRef,
memory_region_keeper: MemoryRegionKeeperRef,
customized_region_lease_renewer: Option<CustomizedRegionLeaseRenewerRef>,
) -> Self {
let region_lease_keeper =
RegionLeaseKeeper::new(table_metadata_manager, memory_region_keeper.clone());
@@ -43,6 +57,7 @@ impl RegionLeaseHandler {
Self {
region_lease_seconds,
region_lease_keeper: Arc::new(region_lease_keeper),
customized_region_lease_renewer,
}
}
}
@@ -56,7 +71,7 @@ impl HeartbeatHandler for RegionLeaseHandler {
async fn handle(
&self,
req: &HeartbeatRequest,
_ctx: &mut Context,
ctx: &mut Context,
acc: &mut HeartbeatAccumulator,
) -> Result<HandleControl> {
let Some(stat) = acc.stat.as_ref() else {
@@ -74,16 +89,18 @@ impl HeartbeatHandler for RegionLeaseHandler {
.renew_region_leases(datanode_id, &regions)
.await?;
let renewed = renewed
.into_iter()
.map(|(region_id, region_role)| {
GrantedRegion {
region_id,
region_role,
}
.into()
})
.collect::<Vec<_>>();
let renewed = if let Some(renewer) = &self.customized_region_lease_renewer {
renewer
.renew(ctx, renewed)
.into_iter()
.map(|region| region.into())
.collect()
} else {
renewed
.into_iter()
.map(|(region_id, region_role)| GrantedRegion::new(region_id, region_role).into())
.collect::<Vec<_>>()
};
acc.region_lease = Some(RegionLease {
regions: renewed,
@@ -102,7 +119,7 @@ mod test {
use std::collections::{HashMap, HashSet};
use std::sync::Arc;
use common_meta::datanode::{RegionStat, Stat};
use common_meta::datanode::{RegionManifestInfo, RegionStat, Stat};
use common_meta::distributed_time_constants;
use common_meta::key::table_route::TableRouteValue;
use common_meta::key::test_utils::new_test_table_info;
@@ -139,6 +156,10 @@ mod test {
manifest_size: 0,
sst_size: 0,
index_size: 0,
region_manifest: RegionManifestInfo::Mito {
manifest_version: 0,
flushed_entry_id: 0,
},
}
}
@@ -198,6 +219,7 @@ mod test {
distributed_time_constants::REGION_LEASE_SECS,
table_metadata_manager.clone(),
opening_region_keeper.clone(),
None,
);
handler.handle(&req, ctx, acc).await.unwrap();
@@ -340,6 +362,7 @@ mod test {
distributed_time_constants::REGION_LEASE_SECS,
table_metadata_manager.clone(),
Default::default(),
None,
);
handler.handle(&req, ctx, acc).await.unwrap();

View File

@@ -49,6 +49,7 @@ mod tests {
use common_meta::cache_invalidator::DummyCacheInvalidator;
use common_meta::key::TableMetadataManager;
use common_meta::kv_backend::memory::MemoryKvBackend;
use common_meta::region_registry::LeaderRegionRegistry;
use common_meta::sequence::SequenceBuilder;
use common_telemetry::tracing_context::W3cTrace;
@@ -84,6 +85,7 @@ mod tests {
is_infancy: false,
table_metadata_manager: Arc::new(TableMetadataManager::new(kv_backend.clone())),
cache_invalidator: Arc::new(DummyCacheInvalidator),
leader_region_registry: Arc::new(LeaderRegionRegistry::new()),
};
let req = HeartbeatRequest {

View File

@@ -17,7 +17,7 @@
#![feature(extract_if)]
pub mod bootstrap;
mod cache_invalidator;
pub mod cache_invalidator;
pub mod cluster;
pub mod election;
pub mod error;
@@ -27,7 +27,7 @@ pub mod handler;
pub mod key;
pub mod lease;
pub mod metasrv;
mod metrics;
pub mod metrics;
#[cfg(feature = "mock")]
pub mod mocks;
pub mod procedure;

View File

@@ -36,6 +36,7 @@ use common_meta::leadership_notifier::{
use common_meta::node_expiry_listener::NodeExpiryListener;
use common_meta::peer::Peer;
use common_meta::region_keeper::MemoryRegionKeeperRef;
use common_meta::region_registry::LeaderRegionRegistryRef;
use common_meta::wal_options_allocator::WalOptionsAllocatorRef;
use common_options::datanode::DatanodeClientOptions;
use common_procedure::options::ProcedureConfig;
@@ -182,6 +183,7 @@ impl Default for MetasrvOptions {
// The etcd the maximum size of any request is 1.5 MiB
// 1500KiB = 1536KiB (1.5MiB) - 36KiB (reserved size of key)
max_metadata_value_size: Some(ReadableSize::kb(1500)),
max_running_procedures: 128,
},
failure_detector: PhiAccrualFailureDetectorOptions::default(),
datanode: DatanodeClientOptions::default(),
@@ -257,11 +259,13 @@ pub struct Context {
pub is_infancy: bool,
pub table_metadata_manager: TableMetadataManagerRef,
pub cache_invalidator: CacheInvalidatorRef,
pub leader_region_registry: LeaderRegionRegistryRef,
}
impl Context {
pub fn reset_in_memory(&self) {
self.in_memory.reset();
self.leader_region_registry.reset();
}
}
@@ -402,6 +406,7 @@ pub struct Metasrv {
region_migration_manager: RegionMigrationManagerRef,
region_supervisor_ticker: Option<RegionSupervisorTickerRef>,
cache_invalidator: CacheInvalidatorRef,
leader_region_registry: LeaderRegionRegistryRef,
plugins: Plugins,
}
@@ -667,6 +672,7 @@ impl Metasrv {
let election = self.election.clone();
let table_metadata_manager = self.table_metadata_manager.clone();
let cache_invalidator = self.cache_invalidator.clone();
let leader_region_registry = self.leader_region_registry.clone();
Context {
server_addr,
@@ -679,6 +685,7 @@ impl Metasrv {
is_infancy: false,
table_metadata_manager,
cache_invalidator,
leader_region_registry,
}
}
}

View File

@@ -34,6 +34,7 @@ use common_meta::kv_backend::memory::MemoryKvBackend;
use common_meta::kv_backend::{KvBackendRef, ResettableKvBackendRef};
use common_meta::node_manager::NodeManagerRef;
use common_meta::region_keeper::MemoryRegionKeeper;
use common_meta::region_registry::LeaderRegionRegistry;
use common_meta::sequence::SequenceBuilder;
use common_meta::state_store::KvStateStore;
use common_meta::wal_options_allocator::build_wal_options_allocator;
@@ -49,7 +50,7 @@ use crate::flow_meta_alloc::FlowPeerAllocator;
use crate::greptimedb_telemetry::get_greptimedb_telemetry_task;
use crate::handler::failure_handler::RegionFailureHandler;
use crate::handler::flow_state_handler::FlowStateHandler;
use crate::handler::region_lease_handler::RegionLeaseHandler;
use crate::handler::region_lease_handler::{CustomizedRegionLeaseRenewerRef, RegionLeaseHandler};
use crate::handler::{HeartbeatHandlerGroupBuilder, HeartbeatMailbox, Pushers};
use crate::lease::MetaPeerLookupService;
use crate::metasrv::{
@@ -325,12 +326,14 @@ impl MetasrvBuilder {
None
};
let leader_region_registry = Arc::new(LeaderRegionRegistry::default());
let ddl_manager = Arc::new(
DdlManager::try_new(
DdlContext {
node_manager,
cache_invalidator: cache_invalidator.clone(),
memory_region_keeper: memory_region_keeper.clone(),
leader_region_registry: leader_region_registry.clone(),
table_metadata_manager: table_metadata_manager.clone(),
table_metadata_allocator: table_metadata_allocator.clone(),
flow_metadata_manager: flow_metadata_manager.clone(),
@@ -343,6 +346,10 @@ impl MetasrvBuilder {
.context(error::InitDdlManagerSnafu)?,
);
let customized_region_lease_renewer = plugins
.as_ref()
.and_then(|plugins| plugins.get::<CustomizedRegionLeaseRenewerRef>());
let handler_group_builder = match handler_group_builder {
Some(handler_group_builder) => handler_group_builder,
None => {
@@ -350,6 +357,7 @@ impl MetasrvBuilder {
distributed_time_constants::REGION_LEASE_SECS,
table_metadata_manager.clone(),
memory_region_keeper.clone(),
customized_region_lease_renewer,
);
HeartbeatHandlerGroupBuilder::new(pushers)
@@ -397,6 +405,7 @@ impl MetasrvBuilder {
region_migration_manager,
region_supervisor_ticker,
cache_invalidator,
leader_region_registry,
})
}
}
@@ -430,6 +439,7 @@ fn build_procedure_manager(
let manager_config = ManagerConfig {
max_retry_times: options.procedure.max_retry_times,
retry_delay: options.procedure.retry_delay,
max_running_procedures: options.procedure.max_running_procedures,
..Default::default()
};
let state_store = KvStateStore::new(kv_backend.clone()).with_max_value_size(

View File

@@ -19,11 +19,12 @@ use common_procedure::ProcedureManagerRef;
use snafu::ResultExt;
pub mod region_migration;
#[cfg(test)]
mod test_util;
#[cfg(any(test, feature = "testing"))]
pub mod test_util;
#[cfg(test)]
mod tests;
pub mod utils;
pub mod wal_prune;
#[derive(Clone)]
pub struct ProcedureManagerListenerAdapter(pub ProcedureManagerRef);

View File

@@ -12,13 +12,26 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::collections::HashMap;
use api::v1::meta::mailbox_message::Payload;
use api::v1::meta::{HeartbeatResponse, MailboxMessage};
use common_meta::instruction::{
DowngradeRegionReply, InstructionReply, SimpleReply, UpgradeRegionReply,
};
use common_meta::key::table_route::TableRouteValue;
use common_meta::key::test_utils::new_test_table_info;
use common_meta::key::TableMetadataManagerRef;
use common_meta::peer::Peer;
use common_meta::region_registry::{
LeaderRegion, LeaderRegionManifestInfo, LeaderRegionRegistryRef,
};
use common_meta::rpc::router::{Region, RegionRoute};
use common_meta::sequence::Sequence;
use common_time::util::current_time_millis;
use common_wal::options::{KafkaWalOptions, WalOptions};
use store_api::logstore::EntryId;
use store_api::storage::RegionId;
use tokio::sync::mpsc::{Receiver, Sender};
use crate::error::Result;
@@ -73,11 +86,7 @@ pub fn send_mock_reply(
}
/// Generates a [InstructionReply::OpenRegion] reply.
pub(crate) fn new_open_region_reply(
id: u64,
result: bool,
error: Option<String>,
) -> MailboxMessage {
pub fn new_open_region_reply(id: u64, result: bool, error: Option<String>) -> MailboxMessage {
MailboxMessage {
id,
subject: "mock".to_string(),
@@ -156,3 +165,94 @@ pub fn new_upgrade_region_reply(
)),
}
}
pub async fn new_wal_prune_metadata(
table_metadata_manager: TableMetadataManagerRef,
leader_region_registry: LeaderRegionRegistryRef,
n_region: u32,
n_table: u32,
offsets: &[i64],
threshold: u64,
topic: String,
) -> (EntryId, Vec<RegionId>) {
let datanode_id = 1;
let from_peer = Peer::empty(datanode_id);
let mut min_last_entry_id = 0;
let mut region_entry_ids = HashMap::with_capacity(n_table as usize * n_region as usize);
for table_id in 0..n_table {
let region_ids = (0..n_region)
.map(|i| RegionId::new(table_id, i))
.collect::<Vec<_>>();
let table_info = new_test_table_info(table_id, 0..n_region).into();
let region_routes = region_ids
.iter()
.map(|region_id| RegionRoute {
region: Region::new_test(*region_id),
leader_peer: Some(from_peer.clone()),
..Default::default()
})
.collect::<Vec<_>>();
let wal_options = WalOptions::Kafka(KafkaWalOptions {
topic: topic.clone(),
});
let wal_options = serde_json::to_string(&wal_options).unwrap();
let region_wal_options: HashMap<u32, String> = (0..n_region)
.map(|region_number| (region_number, wal_options.clone()))
.collect();
table_metadata_manager
.create_table_metadata(
table_info,
TableRouteValue::physical(region_routes),
region_wal_options,
)
.await
.unwrap();
let current_region_entry_ids = region_ids
.iter()
.map(|region_id| {
let rand_n = rand::random::<u64>() as usize;
let current_last_entry_id = offsets[rand_n % offsets.len()] as u64;
min_last_entry_id = min_last_entry_id.min(current_last_entry_id);
(*region_id, current_last_entry_id)
})
.collect::<HashMap<_, _>>();
region_entry_ids.extend(current_region_entry_ids.clone());
update_in_memory_region_last_entry_id(&leader_region_registry, current_region_entry_ids)
.await
.unwrap();
}
let regions_to_flush = region_entry_ids
.iter()
.filter_map(|(region_id, last_entry_id)| {
if last_entry_id - min_last_entry_id > threshold {
Some(*region_id)
} else {
None
}
})
.collect::<Vec<_>>();
(min_last_entry_id, regions_to_flush)
}
pub async fn update_in_memory_region_last_entry_id(
leader_region_registry: &LeaderRegionRegistryRef,
region_entry_ids: HashMap<RegionId, u64>,
) -> Result<()> {
let mut key_values = Vec::with_capacity(region_entry_ids.len());
for (region_id, flushed_entry_id) in region_entry_ids {
let value = LeaderRegion {
datanode_id: 1,
manifest: LeaderRegionManifestInfo::Mito {
manifest_version: 0,
flushed_entry_id,
},
};
key_values.push((region_id, value));
}
leader_region_registry.batch_put(key_values);
Ok(())
}

View File

@@ -123,6 +123,7 @@ pub mod test_data {
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::rpc::router::RegionRoute;
use common_meta::sequence::SequenceBuilder;
use common_meta::wal_options_allocator::WalOptionsAllocator;
@@ -227,6 +228,7 @@ pub mod test_data {
flow_metadata_manager,
flow_metadata_allocator,
memory_region_keeper: Arc::new(MemoryRegionKeeper::new()),
leader_region_registry: Arc::new(LeaderRegionRegistry::default()),
region_failure_detector_controller: Arc::new(NoopRegionFailureDetectorControl),
}
}

View File

@@ -0,0 +1,434 @@
// 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::HashMap;
use std::sync::Arc;
use common_error::ext::BoxedError;
use common_meta::key::TableMetadataManagerRef;
use common_meta::lock_key::RemoteWalLock;
use common_meta::region_registry::LeaderRegionRegistryRef;
use common_procedure::error::ToJsonSnafu;
use common_procedure::{
Context as ProcedureContext, Error as ProcedureError, LockKey, Procedure,
Result as ProcedureResult, Status, StringKey,
};
use common_telemetry::warn;
use log_store::kafka::DEFAULT_PARTITION;
use rskafka::client::partition::UnknownTopicHandling;
use rskafka::client::Client;
use serde::{Deserialize, Serialize};
use snafu::ResultExt;
use store_api::logstore::EntryId;
use store_api::storage::RegionId;
use crate::error::{self, BuildPartitionClientSnafu, DeleteRecordSnafu, TableMetadataManagerSnafu};
use crate::Result;
type KafkaClientRef = Arc<Client>;
const TIMEOUT: i32 = 1000;
/// The state of WAL pruning.
#[derive(Debug, Serialize, Deserialize)]
pub enum WalPruneState {
Prepare,
Prune,
}
pub struct Context {
/// The Kafka client.
client: KafkaClientRef,
/// The table metadata manager.
table_metadata_manager: TableMetadataManagerRef,
leader_region_registry: LeaderRegionRegistryRef,
}
/// The data of WAL pruning.
#[derive(Serialize, Deserialize)]
pub struct WalPruneData {
/// The topic name to prune.
pub topic: String,
/// The minimum flush entry id for topic, which is used to prune the WAL.
/// If the topic has no region, the value is set to `None`.
pub min_flushed_entry_id: EntryId,
/// The state.
pub state: WalPruneState,
}
/// The procedure to prune WAL.
pub struct WalPruneProcedure {
pub data: WalPruneData,
pub context: Context,
}
impl WalPruneProcedure {
const TYPE_NAME: &'static str = "metasrv-procedure::WalPrune";
pub fn new(topic: String, context: Context) -> Self {
Self {
data: WalPruneData {
topic,
min_flushed_entry_id: 0,
state: WalPruneState::Prepare,
},
context,
}
}
pub fn from_json(json: &str, context: Context) -> ProcedureResult<Self> {
let data: WalPruneData = serde_json::from_str(json).context(ToJsonSnafu)?;
Ok(Self { data, context })
}
/// Calculate the last entry id to prune for each topic.
pub async fn on_prepare(&mut self) -> Result<Status> {
let region_ids = self
.context
.table_metadata_manager
.topic_region_manager()
.regions(&self.data.topic)
.await
.context(TableMetadataManagerSnafu)
.map_err(BoxedError::new)
.with_context(|_| error::RetryLaterWithSourceSnafu {
reason: "Failed to get topic-region map",
})?;
let flush_entry_ids_map: HashMap<_, _> = self
.context
.leader_region_registry
.batch_get(region_ids.iter().cloned())
.into_iter()
.map(|(region_id, region)| {
let flushed_entry_id = region.manifest.min_flushed_entry_id();
(region_id, flushed_entry_id)
})
.collect();
if region_ids.is_empty() {
// No regions to prune.
return Ok(Status::done());
}
// Check if the `flush_entry_ids_map` contains all region ids.
let non_collected_region_ids =
check_heartbeat_collected_region_ids(&region_ids, &flush_entry_ids_map);
if !non_collected_region_ids.is_empty() {
// The heartbeat collected region ids do not contain all region ids in the topic-region map.
// In this case, we should not prune the WAL.
warn!("The heartbeat collected region ids do not contain all region ids in the topic-region map. Aborting the WAL prune procedure.
topic: {}, non-collected region ids: {:?}", self.data.topic, non_collected_region_ids);
return Ok(Status::done());
}
// Safety: `flush_entry_ids_map` are not empty.
self.data.min_flushed_entry_id = *(flush_entry_ids_map.values().min().unwrap());
self.data.state = WalPruneState::Prune;
Ok(Status::executing(true))
}
/// Prune the WAL.
pub async fn on_prune(&mut self) -> Result<Status> {
// Safety: last_entry_ids are loaded in on_prepare.
let partition_client = self
.context
.client
.partition_client(
self.data.topic.clone(),
DEFAULT_PARTITION,
UnknownTopicHandling::Retry,
)
.await
.context(BuildPartitionClientSnafu {
topic: self.data.topic.clone(),
partition: DEFAULT_PARTITION,
})?;
partition_client
.delete_records(self.data.min_flushed_entry_id as i64, TIMEOUT)
.await
.context(DeleteRecordSnafu {
topic: self.data.topic.clone(),
partition: DEFAULT_PARTITION,
offset: self.data.min_flushed_entry_id,
})
.map_err(BoxedError::new)
.with_context(|_| error::RetryLaterWithSourceSnafu {
reason: "Failed to delete records",
})?;
// TODO(CookiePie): Persist the minimum flushed entry id to the table metadata manager.
Ok(Status::done())
}
}
#[async_trait::async_trait]
impl Procedure for WalPruneProcedure {
fn type_name(&self) -> &str {
Self::TYPE_NAME
}
fn rollback_supported(&self) -> bool {
false
}
async fn execute(&mut self, _ctx: &ProcedureContext) -> ProcedureResult<Status> {
let state = &self.data.state;
match state {
WalPruneState::Prepare => self.on_prepare().await,
WalPruneState::Prune => self.on_prune().await,
}
.map_err(|e| {
if e.is_retryable() {
ProcedureError::retry_later(e)
} else {
ProcedureError::external(e)
}
})
}
fn dump(&self) -> ProcedureResult<String> {
serde_json::to_string(&self.data).context(ToJsonSnafu)
}
/// WAL prune procedure will read the topic-region map from the table metadata manager,
/// which are modified by `DROP [TABLE|DATABASE]` and `CREATE [TABLE]` operations.
/// But the modifications are atomic, so it does not conflict with the procedure.
/// It only abort the procedure sometimes since the `check_heartbeat_collected_region_ids` fails.
fn lock_key(&self) -> LockKey {
let lock_key: StringKey = RemoteWalLock::Write(self.data.topic.clone()).into();
LockKey::new(vec![lock_key])
}
}
/// Check if the heartbeat collected region ids contains all region ids in the topic-region map.
fn check_heartbeat_collected_region_ids(
region_ids: &[RegionId],
heartbeat_collected_region_ids: &HashMap<RegionId, u64>,
) -> Vec<RegionId> {
let mut non_collected_region_ids = Vec::new();
for region_id in region_ids {
if !heartbeat_collected_region_ids.contains_key(region_id) {
non_collected_region_ids.push(*region_id);
}
}
non_collected_region_ids
}
#[cfg(test)]
mod tests {
use std::assert_matches::assert_matches;
use common_meta::key::TableMetadataManager;
use common_meta::kv_backend::memory::MemoryKvBackend;
use common_meta::region_registry::LeaderRegionRegistry;
use common_meta::wal_options_allocator::build_kafka_topic_creator;
use common_wal::config::kafka::common::{KafkaConnectionConfig, KafkaTopicConfig};
use common_wal::config::kafka::MetasrvKafkaConfig;
use common_wal::test_util::run_test_with_kafka_wal;
use rskafka::record::Record;
use super::*;
use crate::procedure::test_util::new_wal_prune_metadata;
struct TestEnv {
table_metadata_manager: TableMetadataManagerRef,
leader_region_registry: LeaderRegionRegistryRef,
}
impl TestEnv {
fn new() -> Self {
let kv_backend = Arc::new(MemoryKvBackend::new());
let table_metadata_manager = Arc::new(TableMetadataManager::new(kv_backend.clone()));
let leader_region_registry = Arc::new(LeaderRegionRegistry::new());
Self {
table_metadata_manager,
leader_region_registry,
}
}
fn table_metadata_manager(&self) -> &TableMetadataManagerRef {
&self.table_metadata_manager
}
fn leader_region_registry(&self) -> &LeaderRegionRegistryRef {
&self.leader_region_registry
}
}
/// Mock a test env for testing.
/// Including:
/// 1. Create a test env with a mailbox, a table metadata manager and a in-memory kv backend.
/// 2. Prepare some data in the table metadata manager and in-memory kv backend.
/// 3. Generate a `WalPruneProcedure` with the test env.
/// 4. Return the test env, the procedure, the minimum last entry id to prune and the regions to flush.
async fn mock_test_env(
topic: String,
broker_endpoints: Vec<String>,
env: &TestEnv,
) -> (WalPruneProcedure, u64, Vec<RegionId>) {
// Creates a topic manager.
let kafka_topic = KafkaTopicConfig {
replication_factor: broker_endpoints.len() as i16,
..Default::default()
};
let config = MetasrvKafkaConfig {
connection: KafkaConnectionConfig {
broker_endpoints,
..Default::default()
},
kafka_topic,
..Default::default()
};
let topic_creator = build_kafka_topic_creator(&config).await.unwrap();
let table_metadata_manager = env.table_metadata_manager().clone();
let leader_region_registry = env.leader_region_registry().clone();
let offsets = mock_wal_entries(topic_creator.client().clone(), &topic, 10).await;
let (min_last_entry_id, regions_to_flush) = new_wal_prune_metadata(
table_metadata_manager.clone(),
leader_region_registry.clone(),
10,
5,
&offsets,
10,
topic.clone(),
)
.await;
let context = Context {
client: topic_creator.client().clone(),
table_metadata_manager,
leader_region_registry,
};
let wal_prune_procedure = WalPruneProcedure::new(topic, context);
(wal_prune_procedure, min_last_entry_id, regions_to_flush)
}
fn record(i: usize) -> Record {
let key = format!("key_{i}");
let value = format!("value_{i}");
Record {
key: Some(key.into()),
value: Some(value.into()),
timestamp: chrono::Utc::now(),
headers: Default::default(),
}
}
async fn mock_wal_entries(
client: KafkaClientRef,
topic_name: &str,
n_entries: usize,
) -> Vec<i64> {
let controller_client = client.controller_client().unwrap();
let _ = controller_client
.create_topic(topic_name, 1, 1, 5_000)
.await;
let partition_client = client
.partition_client(topic_name, 0, UnknownTopicHandling::Retry)
.await
.unwrap();
let mut offsets = Vec::with_capacity(n_entries);
for i in 0..n_entries {
let record = vec![record(i)];
let offset = partition_client
.produce(
record,
rskafka::client::partition::Compression::NoCompression,
)
.await
.unwrap();
offsets.extend(offset);
}
offsets
}
async fn check_entry_id_existence(
client: KafkaClientRef,
topic_name: &str,
entry_id: i64,
) -> bool {
let partition_client = client
.partition_client(topic_name, 0, UnknownTopicHandling::Retry)
.await
.unwrap();
let (records, _high_watermark) = partition_client
.fetch_records(entry_id, 0..10001, 5_000)
.await
.unwrap();
!records.is_empty()
}
async fn delete_topic(client: KafkaClientRef, topic_name: &str) {
let controller_client = client.controller_client().unwrap();
controller_client
.delete_topic(topic_name, 5_000)
.await
.unwrap();
}
#[tokio::test]
async fn test_procedure_execution() {
run_test_with_kafka_wal(|broker_endpoints| {
Box::pin(async {
common_telemetry::init_default_ut_logging();
let topic_name = "greptime_test_topic".to_string();
let env = TestEnv::new();
let (mut procedure, min_last_entry_id, _) =
mock_test_env(topic_name.clone(), broker_endpoints, &env).await;
// Step 1: Test `on_prepare`.
let status = procedure.on_prepare().await.unwrap();
assert_matches!(status, Status::Executing { persist: true });
assert_matches!(procedure.data.state, WalPruneState::Prune);
assert_eq!(procedure.data.min_flushed_entry_id, min_last_entry_id);
// Step 2: Test `on_prune`.
let status = procedure.on_prune().await.unwrap();
assert_matches!(status, Status::Done { output: None });
// Check if the entry ids after `min_flushed_entry_id` still exist.
assert!(
check_entry_id_existence(
procedure.context.client.clone(),
&topic_name,
procedure.data.min_flushed_entry_id as i64,
)
.await
);
// Check if the entry s before `min_flushed_entry_id` are deleted.
assert!(
procedure.data.min_flushed_entry_id == 0
|| !check_entry_id_existence(
procedure.context.client.clone(),
&topic_name,
procedure.data.min_flushed_entry_id as i64 - 1,
)
.await
);
// `check_heartbeat_collected_region_ids` fails.
// Should log a warning and return `Status::Done`.
procedure.context.leader_region_registry.reset();
let status = procedure.on_prepare().await.unwrap();
assert_matches!(status, Status::Done { output: None });
// Clean up the topic.
delete_topic(procedure.context.client, &topic_name).await;
})
})
.await;
}
}

View File

@@ -94,7 +94,9 @@ impl WeightCompute for RegionNumsBasedWeightCompute {
mod tests {
use std::collections::HashMap;
use common_meta::datanode::{DatanodeStatKey, DatanodeStatValue, RegionStat, Stat};
use common_meta::datanode::{
DatanodeStatKey, DatanodeStatValue, RegionManifestInfo, RegionStat, Stat,
};
use common_meta::peer::Peer;
use store_api::region_engine::RegionRole;
use store_api::storage::RegionId;
@@ -189,6 +191,10 @@ mod tests {
manifest_size: 0,
sst_size: 0,
index_size: 0,
region_manifest: RegionManifestInfo::Mito {
manifest_version: 0,
flushed_entry_id: 0,
},
}],
..Default::default()
}
@@ -210,6 +216,10 @@ mod tests {
manifest_size: 0,
sst_size: 0,
index_size: 0,
region_manifest: RegionManifestInfo::Mito {
manifest_version: 0,
flushed_entry_id: 0,
},
}],
..Default::default()
}
@@ -231,6 +241,10 @@ mod tests {
manifest_size: 0,
sst_size: 0,
index_size: 0,
region_manifest: RegionManifestInfo::Mito {
manifest_version: 0,
flushed_entry_id: 0,
},
}],
..Default::default()
}

View File

@@ -39,8 +39,8 @@ use snafu::ResultExt;
use store_api::metadata::RegionMetadataRef;
use store_api::metric_engine_consts::METRIC_ENGINE_NAME;
use store_api::region_engine::{
RegionEngine, RegionRole, RegionScannerRef, RegionStatistic, SetRegionRoleStateResponse,
SettableRegionRoleState,
RegionEngine, RegionManifestInfo, RegionRole, RegionScannerRef, RegionStatistic,
SetRegionRoleStateResponse, SettableRegionRoleState,
};
use store_api::region_request::{BatchRegionDdlRequest, RegionRequest};
use store_api::storage::{RegionId, ScanRequest, SequenceNumber};
@@ -48,7 +48,7 @@ use store_api::storage::{RegionId, ScanRequest, SequenceNumber};
use self::state::MetricEngineState;
use crate::config::EngineConfig;
use crate::data_region::DataRegion;
use crate::error::{self, Result, UnsupportedRegionRequestSnafu};
use crate::error::{self, MetricManifestInfoSnafu, Result, UnsupportedRegionRequestSnafu};
use crate::metadata_region::MetadataRegion;
use crate::row_modifier::RowModifier;
use crate::utils;
@@ -258,7 +258,29 @@ impl RegionEngine for MetricEngine {
/// Note: Returns `None` if it's a logical region.
fn region_statistic(&self, region_id: RegionId) -> Option<RegionStatistic> {
if self.inner.is_physical_region(region_id) {
self.inner.mito.region_statistic(region_id)
let metadata_region_id = utils::to_metadata_region_id(region_id);
let data_region_id = utils::to_data_region_id(region_id);
let metadata_stat = self.inner.mito.region_statistic(metadata_region_id);
let data_stat = self.inner.mito.region_statistic(data_region_id);
match (metadata_stat, data_stat) {
(Some(metadata_stat), Some(data_stat)) => Some(RegionStatistic {
num_rows: metadata_stat.num_rows + data_stat.num_rows,
memtable_size: metadata_stat.memtable_size + data_stat.memtable_size,
wal_size: metadata_stat.wal_size + data_stat.wal_size,
manifest_size: metadata_stat.manifest_size + data_stat.manifest_size,
sst_size: metadata_stat.sst_size + data_stat.sst_size,
index_size: metadata_stat.index_size + data_stat.index_size,
manifest: RegionManifestInfo::Metric {
data_flushed_entry_id: data_stat.manifest.data_flushed_entry_id(),
data_manifest_version: data_stat.manifest.data_manifest_version(),
metadata_flushed_entry_id: metadata_stat.manifest.data_flushed_entry_id(),
metadata_manifest_version: metadata_stat.manifest.data_manifest_version(),
},
}),
_ => None,
}
} else {
None
}
@@ -285,6 +307,46 @@ impl RegionEngine for MetricEngine {
Ok(())
}
async fn sync_region(
&self,
region_id: RegionId,
manifest_info: RegionManifestInfo,
) -> Result<(), BoxedError> {
if !manifest_info.is_metric() {
return Err(BoxedError::new(
MetricManifestInfoSnafu { region_id }.build(),
));
}
let metadata_region_id = utils::to_metadata_region_id(region_id);
// checked by ensure above
let metadata_manifest_version = manifest_info
.metadata_manifest_version()
.unwrap_or_default();
let metadata_flushed_entry_id = manifest_info
.metadata_flushed_entry_id()
.unwrap_or_default();
let metadata_region_manifest =
RegionManifestInfo::mito(metadata_manifest_version, metadata_flushed_entry_id);
self.inner
.mito
.sync_region(metadata_region_id, metadata_region_manifest)
.await?;
let data_region_id = utils::to_data_region_id(region_id);
let data_manifest_version = manifest_info.data_manifest_version();
let data_flushed_entry_id = manifest_info.data_flushed_entry_id();
let data_region_manifest =
RegionManifestInfo::mito(data_manifest_version, data_flushed_entry_id);
self.inner
.mito
.sync_region(data_region_id, data_region_manifest)
.await?;
Ok(())
}
async fn set_region_role_state_gracefully(
&self,
region_id: RegionId,

View File

@@ -259,6 +259,13 @@ pub enum Error {
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Expected metric manifest info, region: {}", region_id))]
MetricManifestInfo {
region_id: RegionId,
#[snafu(implicit)]
location: Location,
},
}
pub type Result<T, E = Error> = std::result::Result<T, E>;
@@ -309,6 +316,8 @@ impl ErrorExt for Error {
EncodePrimaryKey { source, .. } => source.status_code(),
CollectRecordBatchStream { source, .. } => source.status_code(),
MetricManifestInfo { .. } => StatusCode::Internal,
}
}

View File

@@ -157,12 +157,16 @@ pub async fn open_compaction_region(
checkpoint_distance: mito_config.manifest_checkpoint_distance,
};
RegionManifestManager::open(region_manifest_options, Default::default())
.await?
.context(EmptyRegionDirSnafu {
region_id: req.region_id,
region_dir: req.region_dir.as_str(),
})?
RegionManifestManager::open(
region_manifest_options,
Default::default(),
Default::default(),
)
.await?
.context(EmptyRegionDirSnafu {
region_id: req.region_id,
region_dir: req.region_dir.as_str(),
})?
};
let manifest = manifest_manager.manifest();

View File

@@ -55,6 +55,8 @@ mod row_selector_test;
#[cfg(test)]
mod set_role_state_test;
#[cfg(test)]
mod sync_test;
#[cfg(test)]
mod truncate_test;
use std::any::Any;
@@ -76,10 +78,11 @@ use snafu::{ensure, OptionExt, ResultExt};
use store_api::codec::PrimaryKeyEncoding;
use store_api::logstore::provider::Provider;
use store_api::logstore::LogStore;
use store_api::manifest::ManifestVersion;
use store_api::metadata::RegionMetadataRef;
use store_api::region_engine::{
BatchResponses, RegionEngine, RegionRole, RegionScannerRef, RegionStatistic,
SetRegionRoleStateResponse, SettableRegionRoleState,
BatchResponses, RegionEngine, RegionManifestInfo, RegionRole, RegionScannerRef,
RegionStatistic, SetRegionRoleStateResponse, SettableRegionRoleState,
};
use store_api::region_request::{AffectedRows, RegionOpenRequest, RegionRequest};
use store_api::storage::{RegionId, ScanRequest, SequenceNumber};
@@ -88,7 +91,8 @@ use tokio::sync::{oneshot, Semaphore};
use crate::cache::CacheStrategy;
use crate::config::MitoConfig;
use crate::error::{
InvalidRequestSnafu, JoinSnafu, RecvSnafu, RegionNotFoundSnafu, Result, SerdeJsonSnafu,
InvalidRequestSnafu, JoinSnafu, MitoManifestInfoSnafu, RecvSnafu, RegionNotFoundSnafu, Result,
SerdeJsonSnafu,
};
use crate::manifest::action::RegionEdit;
use crate::metrics::HANDLE_REQUEST_ELAPSED;
@@ -488,6 +492,20 @@ impl EngineInner {
receiver.await.context(RecvSnafu)
}
async fn sync_region(
&self,
region_id: RegionId,
manifest_info: RegionManifestInfo,
) -> Result<ManifestVersion> {
ensure!(manifest_info.is_mito(), MitoManifestInfoSnafu);
let manifest_version = manifest_info.data_manifest_version();
let (request, receiver) =
WorkerRequest::new_sync_region_request(region_id, manifest_version);
self.workers.submit_to_worker(region_id, request).await?;
receiver.await.context(RecvSnafu)?
}
fn role(&self, region_id: RegionId) -> Option<RegionRole> {
self.workers.get_region(region_id).map(|region| {
if region.is_follower() {
@@ -609,6 +627,18 @@ impl RegionEngine for MitoEngine {
.map_err(BoxedError::new)
}
async fn sync_region(
&self,
region_id: RegionId,
manifest_info: RegionManifestInfo,
) -> Result<(), BoxedError> {
self.inner
.sync_region(region_id, manifest_info)
.await
.map_err(BoxedError::new)
.map(|_| ())
}
fn role(&self, region_id: RegionId) -> Option<RegionRole> {
self.inner.role(region_id)
}

View File

@@ -22,7 +22,7 @@ use common_error::ext::ErrorExt;
use common_error::status_code::StatusCode;
use common_recordbatch::RecordBatches;
use datatypes::prelude::ConcreteDataType;
use datatypes::schema::{ColumnSchema, FulltextAnalyzer, FulltextOptions};
use datatypes::schema::{ColumnSchema, FulltextAnalyzer, FulltextBackend, FulltextOptions};
use store_api::metadata::ColumnMetadata;
use store_api::region_engine::{RegionEngine, RegionRole};
use store_api::region_request::{
@@ -90,6 +90,7 @@ fn alter_column_fulltext_options() -> RegionAlterRequest {
enable: true,
analyzer: FulltextAnalyzer::English,
case_sensitive: false,
backend: FulltextBackend::Bloom,
},
},
},
@@ -557,6 +558,7 @@ async fn test_alter_column_fulltext_options() {
enable: true,
analyzer: FulltextAnalyzer::English,
case_sensitive: false,
backend: FulltextBackend::Bloom,
};
let check_fulltext_options = |engine: &MitoEngine, expected: &FulltextOptions| {
let current_fulltext_options = engine

View File

@@ -0,0 +1,247 @@
// 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::assert_matches::assert_matches;
use api::v1::{Rows, SemanticType};
use common_error::ext::ErrorExt;
use common_recordbatch::RecordBatches;
use datatypes::prelude::ConcreteDataType;
use datatypes::schema::ColumnSchema;
use store_api::metadata::ColumnMetadata;
use store_api::region_engine::{RegionEngine, RegionManifestInfo};
use store_api::region_request::{
AddColumn, AddColumnLocation, AlterKind, RegionAlterRequest, RegionOpenRequest, RegionRequest,
};
use store_api::storage::{RegionId, ScanRequest};
use super::MitoEngine;
use crate::config::MitoConfig;
use crate::error::Error;
use crate::test_util::{
build_rows, flush_region, put_rows, rows_schema, CreateRequestBuilder, TestEnv,
};
fn add_tag1() -> RegionAlterRequest {
RegionAlterRequest {
schema_version: 0,
kind: AlterKind::AddColumns {
columns: vec![AddColumn {
column_metadata: ColumnMetadata {
column_schema: ColumnSchema::new(
"tag_1",
ConcreteDataType::string_datatype(),
true,
),
semantic_type: SemanticType::Tag,
column_id: 3,
},
location: Some(AddColumnLocation::First),
}],
},
}
}
async fn scan_check(
engine: &MitoEngine,
region_id: RegionId,
expected: &str,
num_memtable: usize,
num_files: usize,
) {
let request = ScanRequest::default();
let scanner = engine.scanner(region_id, request).unwrap();
assert_eq!(num_memtable, scanner.num_memtables());
assert_eq!(num_files, scanner.num_files());
let stream = scanner.scan().await.unwrap();
let batches = RecordBatches::try_collect(stream).await.unwrap();
assert_eq!(expected, batches.pretty_print().unwrap());
}
#[tokio::test]
async fn test_sync_after_flush_region() {
common_telemetry::init_default_ut_logging();
let mut env = TestEnv::new();
let engine = env.create_engine(MitoConfig::default()).await;
let region_id = RegionId::new(1, 1);
env.get_schema_metadata_manager()
.register_region_table_info(
region_id.table_id(),
"test_table",
"test_catalog",
"test_schema",
None,
env.get_kv_backend(),
)
.await;
let request = CreateRequestBuilder::new().build();
let region_dir = request.region_dir.clone();
let column_schemas = rows_schema(&request);
engine
.handle_request(region_id, RegionRequest::Create(request))
.await
.unwrap();
let rows = Rows {
schema: column_schemas,
rows: build_rows(0, 3),
};
put_rows(&engine, region_id, rows).await;
// Open the region on the follower engine
let follower_engine = env.create_follower_engine(MitoConfig::default()).await;
follower_engine
.handle_request(
region_id,
RegionRequest::Open(RegionOpenRequest {
engine: String::new(),
region_dir,
options: Default::default(),
// Ensure the region is not replayed from the WAL.
skip_wal_replay: true,
}),
)
.await
.unwrap();
flush_region(&engine, region_id, None).await;
// Scan the region on the leader engine
let expected = "\
+-------+---------+---------------------+
| tag_0 | field_0 | ts |
+-------+---------+---------------------+
| 0 | 0.0 | 1970-01-01T00:00:00 |
| 1 | 1.0 | 1970-01-01T00:00:01 |
| 2 | 2.0 | 1970-01-01T00:00:02 |
+-------+---------+---------------------+";
scan_check(&engine, region_id, expected, 0, 1).await;
common_telemetry::info!("Scan the region on the follower engine");
// Scan the region on the follower engine
let expected = "++\n++";
scan_check(&follower_engine, region_id, expected, 0, 0).await;
// Returns error since the max manifest is 1
let manifest_info = RegionManifestInfo::mito(2, 0);
let err = follower_engine
.sync_region(region_id, manifest_info)
.await
.unwrap_err();
let err = err.as_any().downcast_ref::<Error>().unwrap();
assert_matches!(err, Error::InstallManifestTo { .. });
let manifest_info = RegionManifestInfo::mito(1, 0);
follower_engine
.sync_region(region_id, manifest_info)
.await
.unwrap();
common_telemetry::info!("Scan the region on the follower engine after sync");
// Scan the region on the follower engine
let expected = "\
+-------+---------+---------------------+
| tag_0 | field_0 | ts |
+-------+---------+---------------------+
| 0 | 0.0 | 1970-01-01T00:00:00 |
| 1 | 1.0 | 1970-01-01T00:00:01 |
| 2 | 2.0 | 1970-01-01T00:00:02 |
+-------+---------+---------------------+";
scan_check(&follower_engine, region_id, expected, 0, 1).await;
}
#[tokio::test]
async fn test_sync_after_alter_region() {
common_telemetry::init_default_ut_logging();
let mut env = TestEnv::new();
let engine = env.create_engine(MitoConfig::default()).await;
let region_id = RegionId::new(1, 1);
let request = CreateRequestBuilder::new().build();
env.get_schema_metadata_manager()
.register_region_table_info(
region_id.table_id(),
"test_table",
"test_catalog",
"test_schema",
None,
env.get_kv_backend(),
)
.await;
let column_schemas = rows_schema(&request);
let region_dir = request.region_dir.clone();
engine
.handle_request(region_id, RegionRequest::Create(request))
.await
.unwrap();
let rows = Rows {
schema: column_schemas,
rows: build_rows(0, 3),
};
put_rows(&engine, region_id, rows).await;
// Open the region on the follower engine
let follower_engine = env.create_follower_engine(MitoConfig::default()).await;
follower_engine
.handle_request(
region_id,
RegionRequest::Open(RegionOpenRequest {
engine: String::new(),
region_dir,
options: Default::default(),
// Ensure the region is not replayed from the WAL.
skip_wal_replay: true,
}),
)
.await
.unwrap();
let request = add_tag1();
engine
.handle_request(region_id, RegionRequest::Alter(request))
.await
.unwrap();
let expected = "\
+-------+-------+---------+---------------------+
| tag_1 | tag_0 | field_0 | ts |
+-------+-------+---------+---------------------+
| | 0 | 0.0 | 1970-01-01T00:00:00 |
| | 1 | 1.0 | 1970-01-01T00:00:01 |
| | 2 | 2.0 | 1970-01-01T00:00:02 |
+-------+-------+---------+---------------------+";
scan_check(&engine, region_id, expected, 0, 1).await;
let expected = "++\n++";
scan_check(&follower_engine, region_id, expected, 0, 0).await;
// Sync the region from the leader engine to the follower engine
let manifest_info = RegionManifestInfo::mito(2, 0);
follower_engine
.sync_region(region_id, manifest_info)
.await
.unwrap();
let expected = "\
+-------+-------+---------+---------------------+
| tag_1 | tag_0 | field_0 | ts |
+-------+-------+---------+---------------------+
| | 0 | 0.0 | 1970-01-01T00:00:00 |
| | 1 | 1.0 | 1970-01-01T00:00:01 |
| | 2 | 2.0 | 1970-01-01T00:00:02 |
+-------+-------+---------+---------------------+";
scan_check(&follower_engine, region_id, expected, 0, 1).await;
}

View File

@@ -33,7 +33,7 @@ use store_api::storage::RegionId;
use tokio::time::error::Elapsed;
use crate::cache::file_cache::FileType;
use crate::region::{RegionLeaderState, RegionRoleState};
use crate::region::RegionRoleState;
use crate::schedule::remote_job_scheduler::JobId;
use crate::sst::file::FileId;
use crate::worker::WorkerId;
@@ -511,10 +511,10 @@ pub enum Error {
},
#[snafu(display("Region {} is in {:?} state, expect: {:?}", region_id, state, expect))]
RegionLeaderState {
RegionState {
region_id: RegionId,
state: RegionRoleState,
expect: RegionLeaderState,
expect: RegionRoleState,
#[snafu(implicit)]
location: Location,
},
@@ -812,8 +812,8 @@ pub enum Error {
#[snafu(display(
"Failed to install manifest to {}, region: {}, available manifest version: {}, last version: {}",
target_version,
available_version,
region_id,
available_version,
last_version
))]
InstallManifestTo {
@@ -1023,6 +1023,12 @@ pub enum Error {
#[snafu(display("Incompatible WAL provider change. This is typically caused by changing WAL provider in database config file without completely cleaning existing files. Global provider: {}, region provider: {}", global, region))]
IncompatibleWalProviderChange { global: String, region: String },
#[snafu(display("Expected mito manifest info"))]
MitoManifestInfo {
#[snafu(implicit)]
location: Location,
},
}
pub type Result<T, E = Error> = std::result::Result<T, E>;
@@ -1047,11 +1053,10 @@ impl ErrorExt for Error {
use Error::*;
match self {
OpenDal { .. }
| ReadParquet { .. }
| WriteWal { .. }
| ReadWal { .. }
| DeleteWal { .. } => StatusCode::StorageUnavailable,
OpenDal { .. } | ReadParquet { .. } => StatusCode::StorageUnavailable,
WriteWal { source, .. } | ReadWal { source, .. } | DeleteWal { source, .. } => {
source.status_code()
}
CompressObject { .. }
| DecompressObject { .. }
| SerdeJson { .. }
@@ -1100,7 +1105,8 @@ impl ErrorExt for Error {
| ReadDataPart { .. }
| CorruptedEntry { .. }
| BuildEntry { .. }
| Metadata { .. } => StatusCode::Internal,
| Metadata { .. }
| MitoManifestInfo { .. } => StatusCode::Internal,
OpenRegion { source, .. } => source.status_code(),
@@ -1125,8 +1131,8 @@ impl ErrorExt for Error {
CompactRegion { source, .. } => source.status_code(),
CompatReader { .. } => StatusCode::Unexpected,
InvalidRegionRequest { source, .. } => source.status_code(),
RegionLeaderState { .. } | UpdateManifest { .. } => StatusCode::RegionNotReady,
&FlushableRegionState { .. } => StatusCode::RegionNotReady,
RegionState { .. } | UpdateManifest { .. } => StatusCode::RegionNotReady,
FlushableRegionState { .. } => StatusCode::RegionNotReady,
JsonOptions { .. } => StatusCode::InvalidArguments,
EmptyRegionDir { .. } | EmptyManifestDir { .. } => StatusCode::RegionNotFound,
ArrowReader { .. } => StatusCode::StorageUnavailable,

View File

@@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::sync::atomic::AtomicU64;
use std::sync::atomic::{AtomicU64, Ordering};
use std::sync::Arc;
use common_datasource::compression::CompressionType;
@@ -23,6 +23,7 @@ use snafu::{ensure, OptionExt, ResultExt};
use store_api::manifest::{ManifestVersion, MAX_VERSION, MIN_VERSION};
use store_api::metadata::RegionMetadataRef;
use super::storage::is_checkpoint_file;
use crate::error::{
self, InstallManifestToSnafu, NoCheckpointSnafu, NoManifestsSnafu, RegionStoppedSnafu, Result,
};
@@ -115,7 +116,7 @@ pub struct RegionManifestOptions {
#[derive(Debug)]
pub struct RegionManifestManager {
store: ManifestObjectStore,
last_version: ManifestVersion,
last_version: Arc<AtomicU64>,
checkpointer: Checkpointer,
manifest: Arc<RegionManifest>,
stopped: bool,
@@ -127,6 +128,7 @@ impl RegionManifestManager {
metadata: RegionMetadataRef,
options: RegionManifestOptions,
total_manifest_size: Arc<AtomicU64>,
manifest_version: Arc<AtomicU64>,
) -> Result<Self> {
// construct storage
let mut store = ManifestObjectStore::new(
@@ -164,9 +166,10 @@ impl RegionManifestManager {
store.save(version, &action_list.encode()?).await?;
let checkpointer = Checkpointer::new(region_id, options, store.clone(), MIN_VERSION);
manifest_version.store(version, Ordering::Relaxed);
Ok(Self {
store,
last_version: version,
last_version: manifest_version,
checkpointer,
manifest: Arc::new(manifest),
stopped: false,
@@ -179,6 +182,7 @@ impl RegionManifestManager {
pub async fn open(
options: RegionManifestOptions,
total_manifest_size: Arc<AtomicU64>,
manifest_version: Arc<AtomicU64>,
) -> Result<Option<Self>> {
let _t = MANIFEST_OP_ELAPSED
.with_label_values(&["open"])
@@ -263,9 +267,10 @@ impl RegionManifestManager {
store.clone(),
last_checkpoint_version,
);
manifest_version.store(version, Ordering::Relaxed);
Ok(Some(Self {
store,
last_version: version,
last_version: manifest_version,
checkpointer,
manifest: Arc::new(manifest),
stopped: false,
@@ -290,13 +295,14 @@ impl RegionManifestManager {
.with_label_values(&["install_manifest_to"])
.start_timer();
let last_version = self.last_version();
// Case 1: If the target version is less than the current version, return the current version.
if self.last_version >= target_version {
if last_version >= target_version {
debug!(
"Target version {} is less than or equal to the current version {}, region: {}, skip install",
target_version, self.last_version, self.manifest.metadata.region_id
target_version, last_version, self.manifest.metadata.region_id
);
return Ok(self.last_version);
return Ok(last_version);
}
ensure!(
@@ -310,7 +316,7 @@ impl RegionManifestManager {
let mut manifests = self
.store
// Invariant: last_version < target_version.
.fetch_manifests_strict_from(self.last_version + 1, target_version + 1)
.fetch_manifests_strict_from(last_version + 1, target_version + 1)
.await?;
// Case 2: No manifests in range: [current_version+1, target_version+1)
@@ -322,7 +328,7 @@ impl RegionManifestManager {
if manifests.is_empty() {
debug!(
"Manifests are not strict from {}, region: {}, tries to install the last checkpoint",
self.last_version, self.manifest.metadata.region_id
last_version, self.manifest.metadata.region_id
);
let last_version = self.install_last_checkpoint().await?;
// Case 2.1: If the installed checkpoint version is greater than or equal to the target version, return the last version.
@@ -341,14 +347,14 @@ impl RegionManifestManager {
if manifests.is_empty() {
return NoManifestsSnafu {
region_id: self.manifest.metadata.region_id,
start_version: self.last_version + 1,
start_version: last_version + 1,
end_version: target_version + 1,
last_version: self.last_version,
last_version,
}
.fail();
}
debug_assert_eq!(manifests.first().unwrap().0, self.last_version + 1);
debug_assert_eq!(manifests.first().unwrap().0, last_version + 1);
let mut manifest_builder =
RegionManifestBuilder::with_checkpoint(Some(self.manifest.as_ref().clone()));
@@ -384,28 +390,29 @@ impl RegionManifestManager {
region_id: self.manifest.metadata.region_id,
target_version,
available_version: new_manifest.manifest_version,
last_version: self.last_version,
last_version,
}
);
let version = self.last_version;
let version = self.last_version();
self.manifest = Arc::new(new_manifest);
self.last_version = self.manifest.manifest_version;
let last_version = self.set_version(self.manifest.manifest_version);
info!(
"Install manifest changes from {} to {}, region: {}",
version, self.last_version, self.manifest.metadata.region_id
version, last_version, self.manifest.metadata.region_id
);
Ok(self.last_version)
Ok(last_version)
}
/// Installs the last checkpoint.
pub(crate) async fn install_last_checkpoint(&mut self) -> Result<ManifestVersion> {
let last_version = self.last_version();
let Some((checkpoint, checkpoint_size)) = Self::last_checkpoint(&mut self.store).await?
else {
return NoCheckpointSnafu {
region_id: self.manifest.metadata.region_id,
last_version: self.last_version,
last_version,
}
.fail();
};
@@ -414,14 +421,14 @@ impl RegionManifestManager {
.set_checkpoint_file_size(checkpoint.last_version, checkpoint_size);
let builder = RegionManifestBuilder::with_checkpoint(checkpoint.checkpoint);
let manifest = builder.try_build()?;
self.last_version = manifest.manifest_version;
let last_version = self.set_version(manifest.manifest_version);
self.manifest = Arc::new(manifest);
info!(
"Installed region manifest from checkpoint: {}, region: {}",
checkpoint.last_version, self.manifest.metadata.region_id
);
Ok(self.last_version)
Ok(last_version)
}
/// Updates the manifest. Returns the current manifest version number.
@@ -486,7 +493,7 @@ impl RegionManifestManager {
/// It doesn't lock the manifest directory in the object store so the result
/// may be inaccurate if there are concurrent writes.
pub async fn has_update(&self) -> Result<bool> {
let last_version = self.last_version;
let last_version = self.last_version();
let streamer =
self.store
@@ -499,7 +506,7 @@ impl RegionManifestManager {
let need_update = streamer
.try_any(|entry| async move {
let file_name = entry.name();
if is_delta_file(file_name) {
if is_delta_file(file_name) || is_checkpoint_file(file_name) {
let version = file_version(file_name);
if version > last_version {
return true;
@@ -515,8 +522,18 @@ impl RegionManifestManager {
/// Increases last version and returns the increased version.
fn increase_version(&mut self) -> ManifestVersion {
self.last_version += 1;
self.last_version
let previous = self.last_version.fetch_add(1, Ordering::Relaxed);
previous + 1
}
/// Sets the last version.
fn set_version(&mut self, version: ManifestVersion) -> ManifestVersion {
self.last_version.store(version, Ordering::Relaxed);
version
}
fn last_version(&self) -> ManifestVersion {
self.last_version.load(Ordering::Relaxed)
}
/// Fetches the last [RegionCheckpoint] from storage.
@@ -547,8 +564,8 @@ impl RegionManifestManager {
fn validate_manifest(&self, expect: &RegionMetadataRef, last_version: ManifestVersion) {
let manifest = self.manifest();
assert_eq!(manifest.metadata, *expect);
assert_eq!(self.manifest.manifest_version, self.last_version);
assert_eq!(last_version, self.last_version);
assert_eq!(self.manifest.manifest_version, self.last_version());
assert_eq!(last_version, self.last_version());
}
pub fn store(&self) -> ManifestObjectStore {

View File

@@ -30,15 +30,17 @@ use store_api::codec::PrimaryKeyEncoding;
use store_api::logstore::provider::Provider;
use store_api::manifest::ManifestVersion;
use store_api::metadata::RegionMetadataRef;
use store_api::region_engine::{RegionRole, RegionStatistic, SettableRegionRoleState};
use store_api::region_engine::{
RegionManifestInfo, RegionRole, RegionStatistic, SettableRegionRoleState,
};
use store_api::storage::RegionId;
use crate::access_layer::AccessLayerRef;
use crate::error::{
FlushableRegionStateSnafu, RegionLeaderStateSnafu, RegionNotFoundSnafu, RegionTruncatedSnafu,
Result, UpdateManifestSnafu,
FlushableRegionStateSnafu, RegionNotFoundSnafu, RegionStateSnafu, RegionTruncatedSnafu, Result,
UpdateManifestSnafu,
};
use crate::manifest::action::{RegionMetaAction, RegionMetaActionList};
use crate::manifest::action::{RegionManifest, RegionMetaAction, RegionMetaActionList};
use crate::manifest::manager::RegionManifestManager;
use crate::memtable::MemtableBuilderRef;
use crate::region::version::{VersionControlRef, VersionRef};
@@ -289,6 +291,8 @@ impl MitoRegion {
let wal_usage = self.estimated_wal_usage(memtable_usage);
let manifest_usage = self.stats.total_manifest_size();
let num_rows = version.ssts.num_rows() + version.memtables.num_rows();
let manifest_version = self.stats.manifest_version();
let flushed_entry_id = version.flushed_entry_id;
RegionStatistic {
num_rows,
@@ -297,6 +301,10 @@ impl MitoRegion {
manifest_size: manifest_usage,
sst_size: sst_usage,
index_size: index_usage,
manifest: RegionManifestInfo::Mito {
manifest_version,
flushed_entry_id,
},
}
}
@@ -317,10 +325,10 @@ impl MitoRegion {
.state
.compare_exchange(RegionRoleState::Leader(expect), state)
.map_err(|actual| {
RegionLeaderStateSnafu {
RegionStateSnafu {
region_id: self.region_id,
state: actual,
expect,
expect: RegionRoleState::Leader(expect),
}
.build()
})?;
@@ -358,6 +366,21 @@ impl ManifestContext {
self.manifest_manager.read().await.has_update().await
}
/// Installs the manifest changes from the current version to the target version (inclusive).
///
/// Returns installed [RegionManifest].
/// **Note**: This function is not guaranteed to install the target version strictly.
/// The installed version may be greater than the target version.
pub(crate) async fn install_manifest_to(
&self,
version: ManifestVersion,
) -> Result<Arc<RegionManifest>> {
let mut manager = self.manifest_manager.write().await;
manager.install_manifest_to(version).await?;
Ok(manager.manifest())
}
/// Updates the manifest if current state is `expect_state`.
pub(crate) async fn update_manifest(
&self,
@@ -394,10 +417,10 @@ impl ManifestContext {
} else {
ensure!(
current_state == RegionRoleState::Leader(expect_state),
RegionLeaderStateSnafu {
RegionStateSnafu {
region_id: manifest.metadata.region_id,
state: current_state,
expect: expect_state,
expect: RegionRoleState::Leader(expect_state),
}
);
}
@@ -589,15 +612,34 @@ impl RegionMap {
.context(RegionNotFoundSnafu { region_id })?;
ensure!(
region.is_writable(),
RegionLeaderStateSnafu {
RegionStateSnafu {
region_id,
state: region.state(),
expect: RegionLeaderState::Writable,
expect: RegionRoleState::Leader(RegionLeaderState::Writable),
}
);
Ok(region)
}
/// Gets readonly region by region id.
///
/// Returns error if the region does not exist or is writable.
pub(crate) fn follower_region(&self, region_id: RegionId) -> Result<MitoRegionRef> {
let region = self
.get_region(region_id)
.context(RegionNotFoundSnafu { region_id })?;
ensure!(
region.is_follower(),
RegionStateSnafu {
region_id,
state: region.state(),
expect: RegionRoleState::Follower,
}
);
Ok(region)
}
/// Gets region by region id.
///
/// Calls the callback if the region does not exist.
@@ -747,12 +789,17 @@ pub(crate) type OpeningRegionsRef = Arc<OpeningRegions>;
#[derive(Default, Debug, Clone)]
pub(crate) struct ManifestStats {
total_manifest_size: Arc<AtomicU64>,
manifest_version: Arc<AtomicU64>,
}
impl ManifestStats {
fn total_manifest_size(&self) -> u64 {
self.total_manifest_size.load(Ordering::Relaxed)
}
fn manifest_version(&self) -> u64 {
self.manifest_version.load(Ordering::Relaxed)
}
}
#[cfg(test)]

View File

@@ -224,6 +224,7 @@ impl RegionOpener {
metadata.clone(),
region_manifest_options,
self.stats.total_manifest_size.clone(),
self.stats.manifest_version.clone(),
)
.await?;
@@ -352,6 +353,7 @@ impl RegionOpener {
let Some(manifest_manager) = RegionManifestManager::open(
region_manifest_options,
self.stats.total_manifest_size.clone(),
self.stats.manifest_version.clone(),
)
.await?
else {
@@ -529,9 +531,12 @@ impl RegionMetadataLoader {
region_dir,
&self.object_store_manager,
)?;
let Some(manifest_manager) =
RegionManifestManager::open(region_manifest_options, Arc::new(AtomicU64::new(0)))
.await?
let Some(manifest_manager) = RegionManifestManager::open(
region_manifest_options,
Arc::new(AtomicU64::new(0)),
Arc::new(AtomicU64::new(0)),
)
.await?
else {
return Ok(None);
};

View File

@@ -216,6 +216,12 @@ impl VersionControl {
version_data.version.ssts.mark_all_deleted();
version_data.version = new_version;
}
/// Overwrites the current version with a new version.
pub(crate) fn overwrite_current(&self, version: VersionRef) {
let mut version_data = self.data.write().unwrap();
version_data.version = version;
}
}
pub(crate) type VersionControlRef = Arc<VersionControl>;

View File

@@ -31,6 +31,7 @@ use prost::Message;
use smallvec::SmallVec;
use snafu::{ensure, OptionExt, ResultExt};
use store_api::codec::{infer_primary_key_encoding_from_hint, PrimaryKeyEncoding};
use store_api::manifest::ManifestVersion;
use store_api::metadata::{ColumnMetadata, RegionMetadata, RegionMetadataRef};
use store_api::region_engine::{SetRegionRoleStateResponse, SettableRegionRoleState};
use store_api::region_request::{
@@ -565,6 +566,9 @@ pub(crate) enum WorkerRequest {
/// Use [RegionEdit] to edit a region directly.
EditRegion(RegionEditRequest),
/// Keep the manifest of a region up to date.
SyncRegion(RegionSyncRequest),
}
impl WorkerRequest {
@@ -684,6 +688,21 @@ impl WorkerRequest {
receiver,
)
}
pub(crate) fn new_sync_region_request(
region_id: RegionId,
manifest_version: ManifestVersion,
) -> (WorkerRequest, Receiver<Result<ManifestVersion>>) {
let (sender, receiver) = oneshot::channel();
(
WorkerRequest::SyncRegion(RegionSyncRequest {
region_id,
manifest_version,
sender,
}),
receiver,
)
}
}
/// DDL request to a region.
@@ -869,6 +888,13 @@ pub(crate) struct RegionEditResult {
pub(crate) result: Result<()>,
}
#[derive(Debug)]
pub(crate) struct RegionSyncRequest {
pub(crate) region_id: RegionId,
pub(crate) manifest_version: ManifestVersion,
pub(crate) sender: Sender<Result<ManifestVersion>>,
}
#[cfg(test)]
mod tests {
use api::v1::value::ValueData;

View File

@@ -46,6 +46,8 @@ pub(crate) const TYPE_INVERTED_INDEX: &str = "inverted_index";
pub(crate) const TYPE_FULLTEXT_INDEX: &str = "fulltext_index";
pub(crate) const TYPE_BLOOM_FILTER_INDEX: &str = "bloom_filter_index";
const DEFAULT_FULLTEXT_BLOOM_ROW_GRANULARITY: usize = 8096;
/// Output of the index creation.
#[derive(Debug, Clone, Default)]
pub struct IndexOutput {
@@ -292,6 +294,7 @@ impl IndexerBuilderImpl {
&self.intermediate_manager,
&self.metadata,
self.fulltext_index_config.compress,
DEFAULT_FULLTEXT_BLOOM_ROW_GRANULARITY,
mem_limit,
)
.await;

View File

@@ -20,7 +20,7 @@ use std::sync::Arc;
use common_base::range_read::RangeReader;
use common_telemetry::warn;
use index::bloom_filter::applier::BloomFilterApplier;
use index::bloom_filter::applier::{BloomFilterApplier, InListPredicate};
use index::bloom_filter::reader::{BloomFilterReader, BloomFilterReaderImpl};
use object_store::ObjectStore;
use puffin::puffin_manager::cache::PuffinMetadataCacheRef;
@@ -40,7 +40,6 @@ use crate::error::{
use crate::metrics::INDEX_APPLY_ELAPSED;
use crate::sst::file::FileId;
pub use crate::sst::index::bloom_filter::applier::builder::BloomFilterIndexApplierBuilder;
use crate::sst::index::bloom_filter::applier::builder::Predicate;
use crate::sst::index::bloom_filter::INDEX_BLOB_TYPE;
use crate::sst::index::puffin_manager::{BlobReader, PuffinManagerFactory};
use crate::sst::index::TYPE_BLOOM_FILTER_INDEX;
@@ -71,17 +70,20 @@ pub struct BloomFilterIndexApplier {
bloom_filter_index_cache: Option<BloomFilterIndexCacheRef>,
/// Bloom filter predicates.
filters: HashMap<ColumnId, Vec<Predicate>>,
/// For each column, the value will be retained only if it contains __all__ predicates.
predicates: HashMap<ColumnId, Vec<InListPredicate>>,
}
impl BloomFilterIndexApplier {
/// Creates a new `BloomFilterIndexApplier`.
///
/// For each column, the value will be retained only if it contains __all__ predicates.
pub fn new(
region_dir: String,
region_id: RegionId,
object_store: ObjectStore,
puffin_manager_factory: PuffinManagerFactory,
filters: HashMap<ColumnId, Vec<Predicate>>,
predicates: HashMap<ColumnId, Vec<InListPredicate>>,
) -> Self {
Self {
region_dir,
@@ -91,7 +93,7 @@ impl BloomFilterIndexApplier {
puffin_manager_factory,
puffin_metadata_cache: None,
bloom_filter_index_cache: None,
filters,
predicates,
}
}
@@ -148,7 +150,7 @@ impl BloomFilterIndexApplier {
.map(|(i, range)| (*i, vec![range.clone()]))
.collect::<Vec<_>>();
for (column_id, predicates) in &self.filters {
for (column_id, predicates) in &self.predicates {
let blob = match self
.blob_reader(file_id, *column_id, file_size_hint)
.await?
@@ -167,12 +169,12 @@ impl BloomFilterIndexApplier {
BloomFilterReaderImpl::new(blob),
bloom_filter_cache.clone(),
);
self.apply_filters(reader, predicates, &input, &mut output)
self.apply_predicates(reader, predicates, &mut output)
.await
.context(ApplyBloomFilterIndexSnafu)?;
} else {
let reader = BloomFilterReaderImpl::new(blob);
self.apply_filters(reader, predicates, &input, &mut output)
self.apply_predicates(reader, predicates, &mut output)
.await
.context(ApplyBloomFilterIndexSnafu)?;
}
@@ -298,74 +300,27 @@ impl BloomFilterIndexApplier {
.context(PuffinBuildReaderSnafu)
}
async fn apply_filters<R: BloomFilterReader + Send + 'static>(
async fn apply_predicates<R: BloomFilterReader + Send + 'static>(
&self,
reader: R,
predicates: &[Predicate],
input: &[(usize, Range<usize>)],
predicates: &[InListPredicate],
output: &mut [(usize, Vec<Range<usize>>)],
) -> std::result::Result<(), index::bloom_filter::error::Error> {
let mut applier = BloomFilterApplier::new(Box::new(reader)).await?;
for ((_, r), (_, output)) in input.iter().zip(output.iter_mut()) {
for (_, output) in output.iter_mut() {
// All rows are filtered out, skip the search
if output.is_empty() {
continue;
}
for predicate in predicates {
match predicate {
Predicate::InList(in_list) => {
let res = applier.search(&in_list.list, r.clone()).await?;
if res.is_empty() {
output.clear();
break;
}
*output = intersect_ranges(output, &res);
if output.is_empty() {
break;
}
}
}
}
*output = applier.search(predicates, output).await?;
}
Ok(())
}
}
/// 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
}
fn is_blob_not_found(err: &Error) -> bool {
matches!(
err,
@@ -523,55 +478,4 @@ mod tests {
.await;
assert_eq!(res, vec![(0, vec![0..4]), (1, vec![3..5])]);
}
#[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

@@ -20,6 +20,7 @@ use datafusion_expr::expr::InList;
use datafusion_expr::{BinaryExpr, Expr, Operator};
use datatypes::data_type::ConcreteDataType;
use datatypes::value::Value;
use index::bloom_filter::applier::InListPredicate;
use index::Bytes;
use object_store::ObjectStore;
use puffin::puffin_manager::cache::PuffinMetadataCacheRef;
@@ -35,21 +36,6 @@ use crate::sst::index::bloom_filter::applier::BloomFilterIndexApplier;
use crate::sst::index::codec::IndexValueCodec;
use crate::sst::index::puffin_manager::PuffinManagerFactory;
/// Enumerates types of predicates for value filtering.
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum Predicate {
/// Predicate for matching values in a list.
InList(InListPredicate),
}
/// `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 BloomFilterIndexApplierBuilder<'a> {
region_dir: String,
object_store: ObjectStore,
@@ -58,7 +44,7 @@ pub struct BloomFilterIndexApplierBuilder<'a> {
file_cache: Option<FileCacheRef>,
puffin_metadata_cache: Option<PuffinMetadataCacheRef>,
bloom_filter_index_cache: Option<BloomFilterIndexCacheRef>,
output: HashMap<ColumnId, Vec<Predicate>>,
predicates: HashMap<ColumnId, Vec<InListPredicate>>,
}
impl<'a> BloomFilterIndexApplierBuilder<'a> {
@@ -76,7 +62,7 @@ impl<'a> BloomFilterIndexApplierBuilder<'a> {
file_cache: None,
puffin_metadata_cache: None,
bloom_filter_index_cache: None,
output: HashMap::default(),
predicates: HashMap::default(),
}
}
@@ -107,7 +93,7 @@ impl<'a> BloomFilterIndexApplierBuilder<'a> {
self.traverse_and_collect(expr);
}
if self.output.is_empty() {
if self.predicates.is_empty() {
return Ok(None);
}
@@ -116,7 +102,7 @@ impl<'a> BloomFilterIndexApplierBuilder<'a> {
self.metadata.region_id,
self.object_store,
self.puffin_manager_factory,
self.output,
self.predicates,
)
.with_file_cache(self.file_cache)
.with_puffin_metadata_cache(self.puffin_metadata_cache)
@@ -178,14 +164,12 @@ impl<'a> BloomFilterIndexApplierBuilder<'a> {
return Ok(());
};
let value = encode_lit(lit, data_type)?;
// Create bloom filter predicate
let mut set = HashSet::new();
set.insert(value);
let predicate = Predicate::InList(InListPredicate { list: set });
// Add to output predicates
self.output.entry(column_id).or_default().push(predicate);
self.predicates
.entry(column_id)
.or_default()
.push(InListPredicate {
list: HashSet::from([value]),
});
Ok(())
}
@@ -223,12 +207,12 @@ impl<'a> BloomFilterIndexApplierBuilder<'a> {
}
if !valid_predicates.is_empty() {
self.output
self.predicates
.entry(column_id)
.or_default()
.push(Predicate::InList(InListPredicate {
.push(InListPredicate {
list: valid_predicates,
}));
});
}
Ok(())
@@ -245,7 +229,7 @@ impl<'a> BloomFilterIndexApplierBuilder<'a> {
// TODO(ruihang): extract this and the one under inverted_index into a common util mod.
/// Helper function to encode a literal into bytes.
fn encode_lit(lit: &ScalarValue, data_type: ConcreteDataType) -> Result<Vec<u8>> {
fn encode_lit(lit: &ScalarValue, data_type: ConcreteDataType) -> Result<Bytes> {
let value = Value::try_from(lit.clone()).context(ConvertValueSnafu)?;
let mut bytes = vec![];
let field = SortField::new(data_type);
@@ -323,20 +307,18 @@ mod tests {
&metadata,
factory,
);
let exprs = vec![Expr::BinaryExpr(BinaryExpr {
left: Box::new(column("column1")),
op: Operator::Eq,
right: Box::new(string_lit("value1")),
})];
let result = builder.build(&exprs).unwrap();
assert!(result.is_some());
let filters = result.unwrap().filters;
assert_eq!(filters.len(), 1);
let predicates = result.unwrap().predicates;
assert_eq!(predicates.len(), 1);
let column_predicates = filters.get(&1).unwrap();
let column_predicates = predicates.get(&1).unwrap();
assert_eq!(column_predicates.len(), 1);
let expected = encode_lit(
@@ -344,11 +326,7 @@ mod tests {
ConcreteDataType::string_datatype(),
)
.unwrap();
match &column_predicates[0] {
Predicate::InList(p) => {
assert_eq!(p.list.iter().next().unwrap(), &expected);
}
}
assert_eq!(column_predicates[0].list, HashSet::from([expected]));
}
fn int64_lit(i: i64) -> Expr {
@@ -375,15 +353,10 @@ mod tests {
let result = builder.build(&exprs).unwrap();
assert!(result.is_some());
let filters = result.unwrap().filters;
let column_predicates = filters.get(&2).unwrap();
let predicates = result.unwrap().predicates;
let column_predicates = predicates.get(&2).unwrap();
assert_eq!(column_predicates.len(), 1);
match &column_predicates[0] {
Predicate::InList(p) => {
assert_eq!(p.list.len(), 3);
}
}
assert_eq!(column_predicates[0].list.len(), 3);
}
#[test]
@@ -396,7 +369,6 @@ mod tests {
&metadata,
factory,
);
let exprs = vec![Expr::BinaryExpr(BinaryExpr {
left: Box::new(Expr::BinaryExpr(BinaryExpr {
left: Box::new(column("column1")),
@@ -410,14 +382,13 @@ mod tests {
right: Box::new(int64_lit(42)),
})),
})];
let result = builder.build(&exprs).unwrap();
assert!(result.is_some());
let filters = result.unwrap().filters;
assert_eq!(filters.len(), 2);
assert!(filters.contains_key(&1));
assert!(filters.contains_key(&2));
let predicates = result.unwrap().predicates;
assert_eq!(predicates.len(), 2);
assert!(predicates.contains_key(&1));
assert!(predicates.contains_key(&2));
}
#[test]
@@ -451,14 +422,10 @@ mod tests {
let result = builder.build(&exprs).unwrap();
assert!(result.is_some());
let filters = result.unwrap().filters;
assert!(!filters.contains_key(&1)); // Null equality should be ignored
let column2_predicates = filters.get(&2).unwrap();
match &column2_predicates[0] {
Predicate::InList(p) => {
assert_eq!(p.list.len(), 2); // Only non-null values should be included
}
}
let predicates = result.unwrap().predicates;
assert!(!predicates.contains_key(&1)); // Null equality should be ignored
let column2_predicates = predicates.get(&2).unwrap();
assert_eq!(column2_predicates[0].list.len(), 2);
}
#[test]
@@ -471,7 +438,6 @@ mod tests {
&metadata,
factory,
);
let exprs = vec![
// Non-equality operator
Expr::BinaryExpr(BinaryExpr {
@@ -507,7 +473,6 @@ mod tests {
&metadata,
factory,
);
let exprs = vec![
Expr::BinaryExpr(BinaryExpr {
left: Box::new(column("column1")),
@@ -524,8 +489,8 @@ mod tests {
let result = builder.build(&exprs).unwrap();
assert!(result.is_some());
let filters = result.unwrap().filters;
let column_predicates = filters.get(&1).unwrap();
let predicates = result.unwrap().predicates;
let column_predicates = predicates.get(&1).unwrap();
assert_eq!(column_predicates.len(), 2);
}
}

View File

@@ -13,9 +13,11 @@
// limitations under the License.
use std::collections::HashMap;
use std::sync::atomic::AtomicUsize;
use std::sync::Arc;
use common_telemetry::warn;
use datatypes::schema::FulltextAnalyzer;
use datatypes::schema::{FulltextAnalyzer, FulltextBackend};
use index::fulltext_index::create::{
BloomFilterFulltextIndexCreator, FulltextIndexCreator, TantivyFulltextIndexCreator,
};
@@ -33,7 +35,9 @@ use crate::error::{
use crate::read::Batch;
use crate::sst::file::FileId;
use crate::sst::index::fulltext_index::{INDEX_BLOB_TYPE_BLOOM, INDEX_BLOB_TYPE_TANTIVY};
use crate::sst::index::intermediate::IntermediateManager;
use crate::sst::index::intermediate::{
IntermediateLocation, IntermediateManager, TempFileProvider,
};
use crate::sst::index::puffin_manager::SstPuffinWriter;
use crate::sst::index::statistics::{ByteCount, RowCount, Statistics};
use crate::sst::index::TYPE_FULLTEXT_INDEX;
@@ -56,6 +60,7 @@ impl FulltextIndexer {
intermediate_manager: &IntermediateManager,
metadata: &RegionMetadataRef,
compress: bool,
bloom_row_granularity: usize,
mem_limit: usize,
) -> Result<Option<Self>> {
let mut creators = HashMap::new();
@@ -86,11 +91,29 @@ impl FulltextIndexer {
case_sensitive: options.case_sensitive,
};
// TODO(zhongzc): according to fulltext options, choose in the Tantivy flavor or Bloom Filter flavor.
let creator = TantivyFulltextIndexCreator::new(&intm_path, config, mem_limit)
.await
.context(CreateFulltextCreatorSnafu)?;
let inner = AltFulltextCreator::Tantivy(creator);
let inner = match options.backend {
FulltextBackend::Tantivy => {
let creator = TantivyFulltextIndexCreator::new(&intm_path, config, mem_limit)
.await
.context(CreateFulltextCreatorSnafu)?;
AltFulltextCreator::Tantivy(creator)
}
FulltextBackend::Bloom => {
let temp_file_provider = Arc::new(TempFileProvider::new(
IntermediateLocation::new(&metadata.region_id, sst_file_id),
intermediate_manager.clone(),
));
let global_memory_usage = Arc::new(AtomicUsize::new(0));
let creator = BloomFilterFulltextIndexCreator::new(
config,
bloom_row_granularity,
temp_file_provider,
global_memory_usage,
Some(mem_limit),
);
AltFulltextCreator::Bloom(creator)
}
};
creators.insert(
column_id,
@@ -377,6 +400,7 @@ mod tests {
enable: true,
analyzer: FulltextAnalyzer::English,
case_sensitive: true,
backend: FulltextBackend::Tantivy,
})
.unwrap(),
semantic_type: SemanticType::Field,
@@ -392,6 +416,7 @@ mod tests {
enable: true,
analyzer: FulltextAnalyzer::English,
case_sensitive: false,
backend: FulltextBackend::Tantivy,
})
.unwrap(),
semantic_type: SemanticType::Field,
@@ -407,6 +432,7 @@ mod tests {
enable: true,
analyzer: FulltextAnalyzer::Chinese,
case_sensitive: false,
backend: FulltextBackend::Tantivy,
})
.unwrap(),
semantic_type: SemanticType::Field,
@@ -504,6 +530,7 @@ mod tests {
&intm_mgr,
&region_metadata,
true,
8096,
1024,
)
.await

View File

@@ -625,11 +625,16 @@ impl TestEnv {
};
if let Some(metadata) = initial_metadata {
RegionManifestManager::new(metadata, manifest_opts, Default::default())
.await
.map(Some)
RegionManifestManager::new(
metadata,
manifest_opts,
Default::default(),
Default::default(),
)
.await
.map(Some)
} else {
RegionManifestManager::open(manifest_opts, Default::default()).await
RegionManifestManager::open(manifest_opts, Default::default(), Default::default()).await
}
}

View File

@@ -121,6 +121,7 @@ impl SchedulerEnv {
checkpoint_distance: 10,
},
Default::default(),
Default::default(),
)
.await
.unwrap(),

View File

@@ -824,6 +824,10 @@ impl<S: LogStore> RegionWorkerLoop<S> {
WorkerRequest::Stop => {
debug_assert!(!self.running.load(Ordering::Relaxed));
}
WorkerRequest::SyncRegion(req) => {
self.handle_region_sync(req).await;
}
}
}

View File

@@ -27,6 +27,7 @@ use tokio::time::Instant;
use crate::error::{self, Result};
use crate::region::opener::{replay_memtable, RegionOpener};
use crate::region::MitoRegion;
use crate::worker::RegionWorkerLoop;
impl<S: LogStore> RegionWorkerLoop<S> {
@@ -45,34 +46,12 @@ impl<S: LogStore> RegionWorkerLoop<S> {
}
// Note: Currently, We protect the split brain by ensuring the mutable table is empty.
// It's expensive to execute catch-up requests without `set_writable=true` multiple times.
let is_mutable_empty = region.version().memtables.mutable.is_empty();
let version = region.version();
let is_empty_memtable = version.memtables.is_empty();
// Utilizes the short circuit evaluation.
let region = if !is_mutable_empty || region.manifest_ctx.has_update().await? {
let manifest_version = region.manifest_ctx.manifest_version().await;
let flushed_entry_id = region.version_control.current().last_entry_id;
info!("Reopening the region: {region_id}, empty mutable: {is_mutable_empty}, manifest version: {manifest_version}, flushed entry id: {flushed_entry_id}");
let reopened_region = Arc::new(
RegionOpener::new(
region_id,
region.region_dir(),
self.memtable_builder_provider.clone(),
self.object_store_manager.clone(),
self.purge_scheduler.clone(),
self.puffin_manager_factory.clone(),
self.intermediate_manager.clone(),
self.time_provider.clone(),
)
.cache(Some(self.cache_manager.clone()))
.options(region.version().options.clone())?
.skip_wal_replay(true)
.open(&self.config, &self.wal)
.await?,
);
debug_assert!(!reopened_region.is_writable());
self.regions.insert_region(reopened_region.clone());
reopened_region
let region = if !is_empty_memtable || region.manifest_ctx.has_update().await? {
self.reopen_region(&region).await?
} else {
region
};
@@ -124,4 +103,36 @@ impl<S: LogStore> RegionWorkerLoop<S> {
Ok(0)
}
/// Reopens a region.
pub(crate) async fn reopen_region(
&mut self,
region: &Arc<MitoRegion>,
) -> Result<Arc<MitoRegion>> {
let region_id = region.region_id;
let manifest_version = region.manifest_ctx.manifest_version().await;
let flushed_entry_id = region.version_control.current().last_entry_id;
info!("Reopening the region: {region_id}, manifest version: {manifest_version}, flushed entry id: {flushed_entry_id}");
let reopened_region = Arc::new(
RegionOpener::new(
region_id,
region.region_dir(),
self.memtable_builder_provider.clone(),
self.object_store_manager.clone(),
self.purge_scheduler.clone(),
self.puffin_manager_factory.clone(),
self.intermediate_manager.clone(),
self.time_provider.clone(),
)
.cache(Some(self.cache_manager.clone()))
.options(region.version().options.clone())?
.skip_wal_replay(true)
.open(&self.config, &self.wal)
.await?,
);
debug_assert!(!reopened_region.is_writable());
self.regions.insert_region(reopened_region.clone());
Ok(reopened_region)
}
}

View File

@@ -17,6 +17,7 @@
//! It updates the manifest and applies the changes to the region in background.
use std::collections::{HashMap, VecDeque};
use std::sync::Arc;
use common_telemetry::{info, warn};
use store_api::logstore::LogStore;
@@ -29,10 +30,11 @@ use crate::manifest::action::{
RegionChange, RegionEdit, RegionMetaAction, RegionMetaActionList, RegionTruncate,
};
use crate::metrics::WRITE_CACHE_INFLIGHT_DOWNLOAD;
use crate::region::version::VersionBuilder;
use crate::region::{MitoRegionRef, RegionLeaderState, RegionRoleState};
use crate::request::{
BackgroundNotify, OptionOutputTx, RegionChangeResult, RegionEditRequest, RegionEditResult,
TruncateResult, WorkerRequest,
RegionSyncRequest, TruncateResult, WorkerRequest,
};
use crate::sst::location;
use crate::worker::{RegionWorkerLoop, WorkerListener};
@@ -118,6 +120,61 @@ impl<S: LogStore> RegionWorkerLoop<S> {
self.handle_region_stalled_requests(&change_result.region_id)
.await;
}
/// Handles region sync request.
///
/// Updates the manifest to at least the given version.
/// **Note**: The installed version may be greater than the given version.
pub(crate) async fn handle_region_sync(&mut self, request: RegionSyncRequest) {
let region_id = request.region_id;
let sender = request.sender;
let region = match self.regions.follower_region(region_id) {
Ok(region) => region,
Err(e) => {
let _ = sender.send(Err(e));
return;
}
};
let manifest = match region
.manifest_ctx
.install_manifest_to(request.manifest_version)
.await
{
Ok(manifest) => manifest,
Err(e) => {
let _ = sender.send(Err(e));
return;
}
};
let version = region.version();
if !version.memtables.is_empty() {
warn!(
"Region {} memtables is not empty, which should not happen, manifest version: {}",
region.region_id, manifest.manifest_version
);
}
let region_options = version.options.clone();
let new_mutable = Arc::new(
region
.version()
.memtables
.mutable
.new_with_part_duration(version.compaction_time_window),
);
let metadata = manifest.metadata.clone();
let version = VersionBuilder::new(metadata, new_mutable)
.add_files(region.file_purger.clone(), manifest.files.values().cloned())
.flushed_entry_id(manifest.flushed_entry_id)
.flushed_sequence(manifest.flushed_sequence)
.truncated_entry_id(manifest.truncated_entry_id)
.compaction_time_window(manifest.compaction_time_window)
.options(region_options)
.build();
region.version_control.overwrite_current(Arc::new(version));
let _ = sender.send(Ok(manifest.manifest_version));
}
}
impl<S> RegionWorkerLoop<S> {

View File

@@ -25,7 +25,7 @@ use store_api::codec::PrimaryKeyEncoding;
use store_api::logstore::LogStore;
use store_api::storage::RegionId;
use crate::error::{InvalidRequestSnafu, RegionLeaderStateSnafu, RejectWriteSnafu, Result};
use crate::error::{InvalidRequestSnafu, RegionStateSnafu, RejectWriteSnafu, Result};
use crate::metrics::{WRITE_REJECT_TOTAL, WRITE_ROWS_TOTAL, WRITE_STAGE_ELAPSED};
use crate::region::{RegionLeaderState, RegionRoleState};
use crate::region_write_ctx::RegionWriteCtx;
@@ -240,10 +240,10 @@ impl<S> RegionWorkerLoop<S> {
state => {
// The region is not writable.
sender_req.sender.send(
RegionLeaderStateSnafu {
RegionStateSnafu {
region_id,
state,
expect: RegionLeaderState::Writable,
expect: RegionRoleState::Leader(RegionLeaderState::Writable),
}
.fail(),
);

View File

@@ -21,16 +21,19 @@ use api::v1::column_def::options_from_column_schema;
use api::v1::{
set_index, unset_index, AddColumn, AddColumns, AlterDatabaseExpr, AlterTableExpr, Analyzer,
ColumnDataType, ColumnDataTypeExtension, CreateFlowExpr, CreateTableExpr, CreateViewExpr,
DropColumn, DropColumns, ExpireAfter, ModifyColumnType, ModifyColumnTypes, RenameTable,
SemanticType, SetDatabaseOptions, SetFulltext, SetIndex, SetInverted, SetSkipping,
SetTableOptions, SkippingIndexType as PbSkippingIndexType, TableName, UnsetDatabaseOptions,
UnsetFulltext, UnsetIndex, UnsetInverted, UnsetSkipping, UnsetTableOptions,
DropColumn, DropColumns, ExpireAfter, FulltextBackend as PbFulltextBackend, ModifyColumnType,
ModifyColumnTypes, RenameTable, SemanticType, SetDatabaseOptions, SetFulltext, SetIndex,
SetInverted, SetSkipping, SetTableOptions, SkippingIndexType as PbSkippingIndexType, TableName,
UnsetDatabaseOptions, UnsetFulltext, UnsetIndex, UnsetInverted, UnsetSkipping,
UnsetTableOptions,
};
use common_error::ext::BoxedError;
use common_grpc_expr::util::ColumnExpr;
use common_time::Timezone;
use datafusion::sql::planner::object_name_to_table_reference;
use datatypes::schema::{ColumnSchema, FulltextAnalyzer, Schema, SkippingIndexType, COMMENT_KEY};
use datatypes::schema::{
ColumnSchema, FulltextAnalyzer, FulltextBackend, Schema, SkippingIndexType, COMMENT_KEY,
};
use file_engine::FileOptions;
use query::sql::{
check_file_to_table_schema_compatibility, file_column_schemas_to_table,
@@ -581,6 +584,10 @@ pub(crate) fn to_alter_table_expr(
FulltextAnalyzer::Chinese => Analyzer::Chinese.into(),
},
case_sensitive: options.case_sensitive,
backend: match options.backend {
FulltextBackend::Bloom => PbFulltextBackend::Bloom.into(),
FulltextBackend::Tantivy => PbFulltextBackend::Tantivy.into(),
},
})),
},
sql::statements::alter::SetIndexOperation::Inverted { column_name } => SetIndex {

View File

@@ -56,7 +56,7 @@ use query::parser::QueryStatement;
use query::QueryEngineRef;
use session::context::{Channel, QueryContextRef};
use session::table_name::table_idents_to_full_name;
use set::set_query_timeout;
use set::{set_query_timeout, set_read_preference};
use snafu::{ensure, OptionExt, ResultExt};
use sql::statements::copy::{
CopyDatabase, CopyDatabaseArgument, CopyQueryToArgument, CopyTable, CopyTableArgument,
@@ -379,6 +379,8 @@ impl StatementExecutor {
fn set_variables(&self, set_var: SetVariables, query_ctx: QueryContextRef) -> Result<Output> {
let var_name = set_var.variable.to_string().to_uppercase();
match var_name.as_str() {
"READ_PREFERENCE" => set_read_preference(set_var.value, query_ctx)?,
"TIMEZONE" | "TIME_ZONE" => set_timezone(set_var.value, query_ctx)?,
"BYTEA_OUTPUT" => set_bytea_output(set_var.value, query_ctx)?,

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