Compare commits

...

43 Commits

Author SHA1 Message Date
discord9
67a60646b4 chore: rm unwrap
Signed-off-by: discord9 <discord9@163.com>
2025-07-10 10:55:36 +08:00
discord9
1c3bde7e4e docs: explain fast path
Signed-off-by: discord9 <discord9@163.com>
2025-07-10 10:48:38 +08:00
discord9
e045a0dbdf refactor: faster window expr
Signed-off-by: discord9 <discord9@163.com>
2025-07-10 00:00:20 +08:00
LFC
2f765c8fd4 refactor: remove unnecessary args (#6493)
* x

Signed-off-by: luofucong <luofc@foxmail.com>

* refactor: remove unnecessary args

Signed-off-by: luofucong <luofc@foxmail.com>

---------

Signed-off-by: luofucong <luofc@foxmail.com>
2025-07-09 13:23:15 +00:00
shuiyisong
d99cd98c01 fix: skip nan in prom remote write pipeline (#6489)
Signed-off-by: shuiyisong <xixing.sys@gmail.com>
2025-07-09 11:46:07 +00:00
Weny Xu
a858f55257 refactor(meta): separate validation and execution logic in alter logical tables procedure (#6478)
* refactor(meta): separate validation and execution logic in alter logical tables procedure

Signed-off-by: WenyXu <wenymedia@gmail.com>

* chore: apply suggestions from CR

Signed-off-by: WenyXu <wenymedia@gmail.com>

---------

Signed-off-by: WenyXu <wenymedia@gmail.com>
2025-07-09 06:48:27 +00:00
Ning Sun
916967ea59 feat: allow alternative version string (#6472)
* feat: allow alternative version string

* refactor: rename original version function to verbose_version

Signed-off-by: Ning Sun <sunning@greptime.com>

---------

Signed-off-by: Ning Sun <sunning@greptime.com>
2025-07-09 06:43:01 +00:00
Weny Xu
c58d8aa94a refactor(meta): extract AlterTableExecutor from AlterTableProcedure (#6470)
* refactor(meta): extract `AlterTableExecutor` from `AlterTableProcedure`

Signed-off-by: WenyXu <wenymedia@gmail.com>

* chore: apply suggestions from CR

Signed-off-by: WenyXu <wenymedia@gmail.com>

---------

Signed-off-by: WenyXu <wenymedia@gmail.com>
2025-07-09 05:13:19 +00:00
Ning Sun
eeb061ca74 feat: allow float number literal in step (#6483)
* chore: allow float number literal as step

Signed-off-by: Ning Sun <sunning@greptime.com>

* chore: switch to released version of promql parser

Signed-off-by: Ning Sun <sunning@greptime.com>

---------

Signed-off-by: Ning Sun <sunning@greptime.com>
2025-07-09 03:09:09 +00:00
shuiyisong
f7282fde28 chore: sort range query return values (#6474)
* chore: sort range query return values

* chore: add comments

* chore: add is_sorted check

* fix: test
2025-07-09 02:27:12 +00:00
dennis zhuang
a4bd11fb9c fix: empty statements hang (#6480)
* fix: empty statements hang

Signed-off-by: Dennis Zhuang <killme2008@gmail.com>

* tests: add cases

Signed-off-by: Dennis Zhuang <killme2008@gmail.com>

---------

Signed-off-by: Dennis Zhuang <killme2008@gmail.com>
2025-07-09 02:13:14 +00:00
LFC
6dc9e8ddb4 feat: display extension ranges in "explain" (#6475)
* feat: display extension ranges in "explain"

Signed-off-by: luofucong <luofc@foxmail.com>

* fix ci

Signed-off-by: luofucong <luofc@foxmail.com>

---------

Signed-off-by: luofucong <luofc@foxmail.com>
2025-07-09 02:11:23 +00:00
discord9
af03e89139 fix: stricter win sort condition (#6477)
test: sqlness



test: fix sqlness redacted

Signed-off-by: discord9 <discord9@163.com>
2025-07-08 22:27:17 +00:00
jeremyhi
e7a64b7dc0 chore: refactor register_region method to avoid TOCTOU issues (#6468) 2025-07-08 13:26:38 +00:00
Lin Yihai
29739b556e refactor: split some convert function into sql-common crate (#6452)
refactor: split some convert function into `sql-common` crates

Signed-off-by: Yihai Lin <yihai-lin@foxmail.com>
2025-07-08 12:08:33 +00:00
Lei, HUANG
77e50d0e08 chore: expose some config (#6479)
refactor/expose-config:
 ### Make SubCommand and Fields Public in `frontend.rs`

 - Made `subcmd` field in `Command` struct public.
 - Made `SubCommand` enum public.
 - Made `config_file` and `env_prefix` fields in `StartCommand` struct public.

 These changes enhance the accessibility of command-related structures and fields, facilitating external usage and integration.

Signed-off-by: Lei, HUANG <mrsatangel@gmail.com>
2025-07-08 11:52:23 +00:00
LFC
c2f1447345 refactor: stores the http server builder in Metasrv instance (#6461)
* refactor: stores the http server builder in Metasrv instance

Signed-off-by: luofucong <luofc@foxmail.com>

* resolve PR comments

Signed-off-by: luofucong <luofc@foxmail.com>

* fix ci

Signed-off-by: luofucong <luofc@foxmail.com>

---------

Signed-off-by: luofucong <luofc@foxmail.com>
2025-07-07 06:39:05 +00:00
Weny Xu
30f7955d2b feat: add column metadata to response extensions (#6451)
Signed-off-by: WenyXu <wenymedia@gmail.com>
2025-07-07 03:38:13 +00:00
fys
3508fddd74 feat: support show triggers (#6465)
* feat: support show triggers

* add enterprise feature

* chore: remove unused error
2025-07-07 03:30:57 +00:00
Weny Xu
351c741c70 fix(metric-engine): handle stale metadata region recovery failures (#6395)
* fix(metric-engine): handle stale metadata region recovery failures

Signed-off-by: WenyXu <wenymedia@gmail.com>

* test: add unit tests

Signed-off-by: WenyXu <wenymedia@gmail.com>

---------

Signed-off-by: WenyXu <wenymedia@gmail.com>
2025-07-07 03:30:40 +00:00
liyang
bb43d604a4 ci: use ubuntu 16 core machine in release-cn-artifacts (#6464)
ci: use ubuntu 16 core machine
2025-07-04 18:06:07 +00:00
Lei, HUANG
9576bcb9ae fix: filter empty batch in bulk insert api (#6459)
* fix/filter-empty-batch-in-bulk-insert-api:
 **Add Early Return for Empty Record Batches in `bulk_insert.rs`**

 - Implemented an early return in the `Inserter` implementation to handle cases where `record_batch.num_rows()` is zero, improving efficiency by avoiding unnecessary processing.

Signed-off-by: Lei, HUANG <mrsatangel@gmail.com>

* fix/filter-empty-batch-in-bulk-insert-api:
 **Improve Bulk Insert Handling**

 - **`handle_bulk_insert.rs`**: Added a check to handle cases where the batch has zero rows, immediately returning and sending a success response with zero rows processed.
 - **`bulk_insert.rs`**: Enhanced logic to skip processing for masks that select none, optimizing the bulk insert operation by avoiding unnecessary iterations.

 These changes improve the efficiency and robustness of the bulk insert process by handling edge cases more effectively.

Signed-off-by: Lei, HUANG <mrsatangel@gmail.com>

* fix/filter-empty-batch-in-bulk-insert-api:
 ### Refactor and Error Handling Enhancements

 - **Refactored Timestamp Handling**: Introduced `timestamp_array_to_primitive` function in `timestamp.rs` to streamline conversion of timestamp arrays to primitive arrays, reducing redundancy in `handle_bulk_insert.rs` and `bulk_insert.rs`.
 - **Error Handling**: Added `InconsistentTimestampLength` error in `error.rs` to handle mismatched timestamp column lengths in bulk insert operations.
 - **Bulk Insert Logic**: Updated `handle_bulk_insert.rs` to utilize the new timestamp conversion function and added checks for timestamp length consistency.

Signed-off-by: Lei, HUANG <mrsatangel@gmail.com>

* fix/filter-empty-batch-in-bulk-insert-api:
 **Refactor `bulk_insert.rs` to streamline imports**

 - Simplified import statements by removing unused timestamp-related arrays and data types from the `arrow` crate in `bulk_insert.rs`.

Signed-off-by: Lei, HUANG <mrsatangel@gmail.com>

---------

Signed-off-by: Lei, HUANG <mrsatangel@gmail.com>
2025-07-04 13:32:10 +00:00
Zhenchi
dc17e6e517 fix: add backward compatibility for SkippingIndexOptions deserialization (#6458)
* fix: add backward compatibility for `SkippingIndexOptions` deserialization

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

* address comments

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

* address comments

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

---------

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>
2025-07-04 11:58:27 +00:00
Yiran
563d25ee04 fix: doc links (#6305)
Signed-off-by: Yiran <cuiyiran3@gmail.com>
2025-07-04 09:52:47 +00:00
Weny Xu
7d17782fd5 feat: persist column ids in table metadata (#6457)
* feat: persist column ids in table metadata

Signed-off-by: WenyXu <wenymedia@gmail.com>

* chore: apply suggestions from CR

Signed-off-by: WenyXu <wenymedia@gmail.com>

---------

Signed-off-by: WenyXu <wenymedia@gmail.com>
2025-07-04 08:12:29 +00:00
fys
c5360601f5 feat: information table extension (#6434)
* feat: information table extension

* avoid use std HashMap behind cfg feature
2025-07-04 04:37:36 +00:00
discord9
9b5baa965c feat: truly limit time range by split window (#6295)
* feat: actually split window to limit time range

feat: truly limit time range by split window

Update src/flow/src/batching_mode/state.rs

Co-authored-by: Lei, HUANG <6406592+v0y4g3r@users.noreply.github.com>
Signed-off-by: discord9 <discord9@163.com>

* chore: added stalled time window range

Signed-off-by: discord9 <discord9@163.com>

* fix: not flush all time range as too expensive

Signed-off-by: discord9 <discord9@163.com>

* test: make it more robust

Signed-off-by: discord9 <discord9@163.com>

* what

Signed-off-by: discord9 <discord9@163.com>

* feat: denfensively handle surplus

Signed-off-by: discord9 <discord9@163.com>

* refactor: per review,explain flush flow

Signed-off-by: discord9 <discord9@163.com>

* chore: per bugbot

Signed-off-by: discord9 <discord9@163.com>

* fix: a temp fix to make mirror insert go first(still need better fix to sync with mirror insert that happens before

Signed-off-by: discord9 <discord9@163.com>

* chore: add todo

Signed-off-by: discord9 <discord9@163.com>

---------

Signed-off-by: discord9 <discord9@163.com>
Co-authored-by: Lei, HUANG <6406592+v0y4g3r@users.noreply.github.com>
2025-07-04 03:37:43 +00:00
Yingwen
76a5145def fix: enable max_execution time for other read only statements (#6454)
Also disable the timeout when timeout is 0

Signed-off-by: evenyag <realevenyag@gmail.com>
2025-07-03 13:46:02 +00:00
Ruihang Xia
7b2703760b feat: skip rule checker on ingestion (#6453)
Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2025-07-03 13:31:16 +00:00
Ruihang Xia
81ea172ce4 feat!: point matrix based partition rule checker (#6431)
* bare implementation

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

* stateful generator

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

* error report

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

* fix remap checkpoint

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

* use matrix generator as iterator

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

* pre-calculate suffix product

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

* update existing test cases

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

* fix clippy

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

* sqlness

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

* fix ut

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

* clean up

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

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2025-07-03 06:50:02 +00:00
dennis zhuang
f7c363f969 fix: label_replace and label_join functions when used as sub‐expressions (#6443)
* fix: label_replace and label_join functions in expressions

Signed-off-by: Dennis Zhuang <killme2008@gmail.com>

* chore: remove update_fields

Signed-off-by: Dennis Zhuang <killme2008@gmail.com>

* chore: tql eval -> TQL EVAL

Signed-off-by: Dennis Zhuang <killme2008@gmail.com>

* fix: empty regex and not existing source label

Signed-off-by: Dennis Zhuang <killme2008@gmail.com>

* chore: simplfy test

Signed-off-by: Dennis Zhuang <killme2008@gmail.com>

* fix: test

Signed-off-by: Dennis Zhuang <killme2008@gmail.com>

* fix: test

Signed-off-by: Dennis Zhuang <killme2008@gmail.com>

---------

Signed-off-by: Dennis Zhuang <killme2008@gmail.com>
2025-07-03 05:34:22 +00:00
Ruihang Xia
5f2daae087 fix: remap column indices on overriding logical table partitions (#6446)
* fix: remap column indices on overriding logical table partitions

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

* sqlness

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

* refactor map query

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

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2025-07-02 12:12:00 +00:00
Yingwen
b1b0d0136f fix: correct MAX_EXECUTION_TIME timeout calculation (#6444)
* feat: implement statement timeout in frontend instance

Signed-off-by: evenyag <realevenyag@gmail.com>

* fix: fail fast when timeout is 0

Signed-off-by: evenyag <realevenyag@gmail.com>

* fix: update start time

Signed-off-by: evenyag <realevenyag@gmail.com>

---------

Signed-off-by: evenyag <realevenyag@gmail.com>
2025-07-02 08:31:40 +00:00
Zhenchi
599f289f59 feat: add granularity and false_positive_rate options for indexes (#6416)
* feat: add `granularity` and `false_positive_rate` options for indexes

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

* address comments

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

* upgrade proto

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

---------

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>
2025-07-02 07:33:39 +00:00
LFC
385f12a62e refactor: extract the common method for errors into tonic status (#6437)
Signed-off-by: luofucong <luofc@foxmail.com>
2025-07-02 02:57:30 +00:00
fys
6b90e2b6b4 fix: allow clippy::print_stdout in cli crate (#6436)
* fix: allow clippy::print_stdout in cli crate

* add clippy lint options
2025-07-02 01:40:58 +00:00
ZonaHe
a4f3e96e96 feat: update dashboard to v0.10.2 (#6433)
Co-authored-by: sunchanglong <sunchanglong@users.noreply.github.com>
2025-07-02 01:27:37 +00:00
Ruihang Xia
2b0f27da51 feat: don't allow creating flow with the same sink and source table (#6435)
Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2025-07-01 11:33:09 +00:00
Weny Xu
e0382eeb7c fix: fix dest_keys chunks bug in TombstoneManager (#6432)
* fix(meta): fix dest_keys_chunks bug in TombstoneManager

Signed-off-by: WenyXu <wenymedia@gmail.com>

* chore: fix typo

Signed-off-by: WenyXu <wenymedia@gmail.com>

* fix: fix sqlness tests

Signed-off-by: WenyXu <wenymedia@gmail.com>

---------

Signed-off-by: WenyXu <wenymedia@gmail.com>
2025-07-01 09:20:13 +00:00
liyang
4aa6add8dc ci: add check-version script to check whether push the latast image (#6415)
Signed-off-by: liyang <daviderli614@gmail.com>
2025-07-01 07:45:47 +00:00
zyy17
645988975e refactor: add RegionMigrationTriggerReason in RegionMigrationProcedureTask (#6413)
Signed-off-by: zyy17 <zyylsxm@gmail.com>
2025-06-30 11:16:41 +00:00
LFC
a203909de3 feat: extension range definition (#6386)
* feat: defined extension range

Signed-off-by: luofucong <luofc@foxmail.com>

* remove feature parameters

Signed-off-by: luofucong <luofc@foxmail.com>

* resolve PR comments

Signed-off-by: luofucong <luofc@foxmail.com>

* resolve PR comments

Signed-off-by: luofucong <luofc@foxmail.com>

---------

Signed-off-by: luofucong <luofc@foxmail.com>
2025-06-30 02:42:40 +00:00
discord9
616e76941a feat: flow query parallel=1&query faster with many windows&min one time window (#6324)
* feat: flow query parallel=1&query faster when
windows too many&min one time window

Signed-off-by: discord9 <discord9@163.com>

* chore: default flow query parallelism=1

Signed-off-by: discord9 <discord9@163.com>

* refactor: use query options in flownode per review

Signed-off-by: discord9 <discord9@163.com>

* docs: update comment

Signed-off-by: discord9 <discord9@163.com>

* chore: fix test

Signed-off-by: discord9 <discord9@163.com>

* chore: per review

Signed-off-by: discord9 <discord9@163.com>

* chore: make config docs

Signed-off-by: discord9 <discord9@163.com>

---------

Signed-off-by: discord9 <discord9@163.com>
2025-06-30 02:17:01 +00:00
235 changed files with 8733 additions and 4004 deletions

View File

@@ -12,3 +12,6 @@ fetch = true
checkout = true
list_files = true
internal_use_git2 = false
[env]
CARGO_WORKSPACE_DIR = { value = "", relative = true }

42
.github/scripts/check-version.sh vendored Executable file
View File

@@ -0,0 +1,42 @@
#!/bin/bash
# Get current version
CURRENT_VERSION=$1
if [ -z "$CURRENT_VERSION" ]; then
echo "Error: Failed to get current version"
exit 1
fi
# Get the latest version from GitHub Releases
API_RESPONSE=$(curl -s "https://api.github.com/repos/GreptimeTeam/greptimedb/releases/latest")
if [ -z "$API_RESPONSE" ] || [ "$(echo "$API_RESPONSE" | jq -r '.message')" = "Not Found" ]; then
echo "Error: Failed to fetch latest version from GitHub"
exit 1
fi
# Get the latest version
LATEST_VERSION=$(echo "$API_RESPONSE" | jq -r '.tag_name')
if [ -z "$LATEST_VERSION" ] || [ "$LATEST_VERSION" = "null" ]; then
echo "Error: No valid version found in GitHub releases"
exit 1
fi
# Cleaned up version number format (removed possible 'v' prefix and -nightly suffix)
CLEAN_CURRENT=$(echo "$CURRENT_VERSION" | sed 's/^v//' | sed 's/-nightly-.*//')
CLEAN_LATEST=$(echo "$LATEST_VERSION" | sed 's/^v//' | sed 's/-nightly-.*//')
echo "Current version: $CLEAN_CURRENT"
echo "Latest release version: $CLEAN_LATEST"
# Use sort -V to compare versions
HIGHER_VERSION=$(printf "%s\n%s" "$CLEAN_CURRENT" "$CLEAN_LATEST" | sort -V | tail -n1)
if [ "$HIGHER_VERSION" = "$CLEAN_CURRENT" ]; then
echo "Current version ($CLEAN_CURRENT) is NEWER than or EQUAL to latest ($CLEAN_LATEST)"
echo "should-push-latest-tag=true" >> $GITHUB_OUTPUT
else
echo "Current version ($CLEAN_CURRENT) is OLDER than latest ($CLEAN_LATEST)"
echo "should-push-latest-tag=false" >> $GITHUB_OUTPUT
fi

View File

@@ -110,6 +110,8 @@ jobs:
# The 'version' use as the global tag name of the release workflow.
version: ${{ steps.create-version.outputs.version }}
should-push-latest-tag: ${{ steps.check-version.outputs.should-push-latest-tag }}
steps:
- name: Checkout
uses: actions/checkout@v4
@@ -135,6 +137,11 @@ jobs:
GITHUB_REF_NAME: ${{ github.ref_name }}
NIGHTLY_RELEASE_PREFIX: ${{ env.NIGHTLY_RELEASE_PREFIX }}
- name: Check version
id: check-version
run: |
./.github/scripts/check-version.sh "${{ steps.create-version.outputs.version }}"
- name: Allocate linux-amd64 runner
if: ${{ inputs.build_linux_amd64_artifacts || github.event_name == 'push' || github.event_name == 'schedule' }}
uses: ./.github/actions/start-runner
@@ -314,7 +321,7 @@ jobs:
image-registry-username: ${{ secrets.DOCKERHUB_USERNAME }}
image-registry-password: ${{ secrets.DOCKERHUB_TOKEN }}
version: ${{ needs.allocate-runners.outputs.version }}
push-latest-tag: ${{ github.ref_type == 'tag' && !contains(github.ref_name, 'nightly') && github.event_name != 'schedule' }}
push-latest-tag: ${{ needs.allocate-runners.outputs.should-push-latest-tag == 'true' && github.ref_type == 'tag' && !contains(github.ref_name, 'nightly') && github.event_name != 'schedule' }}
- name: Set build image result
id: set-build-image-result
@@ -332,7 +339,7 @@ jobs:
build-windows-artifacts,
release-images-to-dockerhub,
]
runs-on: ubuntu-latest
runs-on: ubuntu-latest-16-cores
# When we push to ACR, it's easy to fail due to some unknown network issues.
# However, we don't want to fail the whole workflow because of this.
# The ACR have daily sync with DockerHub, so don't worry about the image not being updated.
@@ -361,7 +368,7 @@ jobs:
dev-mode: false
upload-to-s3: true
update-version-info: true
push-latest-tag: ${{ github.ref_type == 'tag' && !contains(github.ref_name, 'nightly') && github.event_name != 'schedule' }}
push-latest-tag: ${{ needs.allocate-runners.outputs.should-push-latest-tag == 'true' && github.ref_type == 'tag' && !contains(github.ref_name, 'nightly') && github.event_name != 'schedule' }}
publish-github-release:
name: Create GitHub release and upload artifacts

53
Cargo.lock generated
View File

@@ -1602,6 +1602,17 @@ version = "1.0.7"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "acbc26382d871df4b7442e3df10a9402bf3cf5e55cbd66f12be38861425f0564"
[[package]]
name = "cargo-manifest"
version = "0.19.1"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "a1d8af896b707212cd0e99c112a78c9497dd32994192a463ed2f7419d29bd8c6"
dependencies = [
"serde",
"thiserror 2.0.12",
"toml 0.8.19",
]
[[package]]
name = "cast"
version = "0.3.0"
@@ -1975,7 +1986,6 @@ dependencies = [
"common-version",
"common-wal",
"datatypes",
"either",
"etcd-client",
"futures",
"humantime",
@@ -2103,7 +2113,6 @@ dependencies = [
"common-wal",
"datanode",
"datatypes",
"either",
"etcd-client",
"file-engine",
"flow",
@@ -2670,12 +2679,31 @@ dependencies = [
"strum 0.27.1",
]
[[package]]
name = "common-sql"
version = "0.16.0"
dependencies = [
"common-base",
"common-datasource",
"common-decimal",
"common-error",
"common-macro",
"common-time",
"datafusion-sql",
"datatypes",
"hex",
"jsonb",
"snafu 0.8.5",
"sqlparser 0.54.0 (git+https://github.com/GreptimeTeam/sqlparser-rs.git?rev=0cf6c04490d59435ee965edd2078e8855bd8471e)",
]
[[package]]
name = "common-telemetry"
version = "0.16.0"
dependencies = [
"backtrace",
"common-error",
"common-version",
"console-subscriber",
"greptime-proto",
"humantime-serde",
@@ -2733,6 +2761,7 @@ name = "common-version"
version = "0.16.0"
dependencies = [
"build-data",
"cargo-manifest",
"const_format",
"serde",
"shadow-rs",
@@ -4201,9 +4230,9 @@ dependencies = [
[[package]]
name = "either"
version = "1.13.0"
version = "1.15.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "60b1af1c220855b6ceac025d3f6ecdd2b7c4894bfe9cd9bda4fbb4bc7c0d4cf0"
checksum = "48c757948c5ede0e46177b7add2e67155f70e33c07fea8284df6576da70b3719"
dependencies = [
"serde",
]
@@ -4702,6 +4731,7 @@ version = "0.16.0"
dependencies = [
"api",
"arc-swap",
"async-stream",
"async-trait",
"auth",
"bytes",
@@ -5147,7 +5177,7 @@ dependencies = [
[[package]]
name = "greptime-proto"
version = "0.1.0"
source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=464226cf8a4a22696503536a123d0b9e318582f4#464226cf8a4a22696503536a123d0b9e318582f4"
source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=ceb1af4fa9309ce65bda0367db7b384df2bb4d4f#ceb1af4fa9309ce65bda0367db7b384df2bb4d4f"
dependencies = [
"prost 0.13.5",
"serde",
@@ -6698,7 +6728,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "4979f22fdb869068da03c9f7528f8297c6fd2606bc3a4affe42e6a823fdb8da4"
dependencies = [
"cfg-if",
"windows-targets 0.48.5",
"windows-targets 0.52.6",
]
[[package]]
@@ -7171,6 +7201,7 @@ dependencies = [
"deadpool",
"deadpool-postgres",
"derive_builder 0.20.1",
"either",
"etcd-client",
"futures",
"h2 0.3.26",
@@ -7242,6 +7273,7 @@ dependencies = [
"common-base",
"common-error",
"common-macro",
"common-meta",
"common-query",
"common-recordbatch",
"common-runtime",
@@ -8456,6 +8488,7 @@ dependencies = [
"common-query",
"common-recordbatch",
"common-runtime",
"common-sql",
"common-telemetry",
"common-test-util",
"common-time",
@@ -9521,8 +9554,9 @@ dependencies = [
[[package]]
name = "promql-parser"
version = "0.5.1"
source = "git+https://github.com/GreptimeTeam/promql-parser.git?rev=0410e8b459dda7cb222ce9596f8bf3971bd07bd2#0410e8b459dda7cb222ce9596f8bf3971bd07bd2"
version = "0.6.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "328fe69c2443ec4f8e6c33ea925dde04a1026e6c95928e89ed02343944cac9bf"
dependencies = [
"cfgrammar",
"chrono",
@@ -11242,6 +11276,7 @@ dependencies = [
"common-recordbatch",
"common-runtime",
"common-session",
"common-sql",
"common-telemetry",
"common-test-util",
"common-time",
@@ -11680,6 +11715,7 @@ dependencies = [
"common-error",
"common-macro",
"common-query",
"common-sql",
"common-time",
"datafusion",
"datafusion-common",
@@ -13154,6 +13190,7 @@ version = "0.8.19"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "a1ed1f98e3fdc28d6d910e6737ae6ab1a93bf1985935a1193e68f93eeb68d24e"
dependencies = [
"indexmap 2.9.0",
"serde",
"serde_spanned",
"toml_datetime",

View File

@@ -30,6 +30,7 @@ members = [
"src/common/recordbatch",
"src/common/runtime",
"src/common/session",
"src/common/sql",
"src/common/stat",
"src/common/substrait",
"src/common/telemetry",
@@ -76,6 +77,8 @@ edition = "2021"
license = "Apache-2.0"
[workspace.lints]
clippy.print_stdout = "warn"
clippy.print_stderr = "warn"
clippy.dbg_macro = "warn"
clippy.implicit_clone = "warn"
clippy.result_large_err = "allow"
@@ -131,11 +134,12 @@ deadpool = "0.12"
deadpool-postgres = "0.14"
derive_builder = "0.20"
dotenv = "0.15"
either = "1.15"
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 = "464226cf8a4a22696503536a123d0b9e318582f4" }
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "ceb1af4fa9309ce65bda0367db7b384df2bb4d4f" }
hex = "0.4"
http = "1"
humantime = "2.1"
@@ -168,9 +172,7 @@ parquet = { version = "54.2", default-features = false, features = ["arrow", "as
paste = "1.0"
pin-project = "1.0"
prometheus = { version = "0.13.3", features = ["process"] }
promql-parser = { git = "https://github.com/GreptimeTeam/promql-parser.git", rev = "0410e8b459dda7cb222ce9596f8bf3971bd07bd2", features = [
"ser",
] }
promql-parser = { version = "0.6", features = ["ser"] }
prost = { version = "0.13", features = ["no-recursion-limit"] }
raft-engine = { version = "0.4.1", default-features = false }
rand = "0.9"
@@ -259,6 +261,7 @@ 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-sql = { path = "src/common/sql" }
common-telemetry = { path = "src/common/telemetry" }
common-test-util = { path = "src/common/test-util" }
common-time = { path = "src/common/time" }

View File

@@ -75,9 +75,9 @@
| --------- | ----------- |
| [Unified Observability Data](https://docs.greptime.com/user-guide/concepts/why-greptimedb) | Store metrics, logs, and traces as timestamped, contextual wide events. Query via [SQL](https://docs.greptime.com/user-guide/query-data/sql), [PromQL](https://docs.greptime.com/user-guide/query-data/promql), and [streaming](https://docs.greptime.com/user-guide/flow-computation/overview). |
| [High Performance & Cost Effective](https://docs.greptime.com/user-guide/manage-data/data-index) | Written in Rust, with a distributed query engine, [rich indexing](https://docs.greptime.com/user-guide/manage-data/data-index), and optimized columnar storage, delivering sub-second responses at PB scale. |
| [Cloud-Native Architecture](https://docs.greptime.com/user-guide/concepts/architecture) | Designed for [Kubernetes](https://docs.greptime.com/user-guide/deployments/deploy-on-kubernetes/greptimedb-operator-management), with compute/storage separation, native object storage (AWS S3, Azure Blob, etc.) and seamless cross-cloud access. |
| [Cloud-Native Architecture](https://docs.greptime.com/user-guide/concepts/architecture) | Designed for [Kubernetes](https://docs.greptime.com/user-guide/deployments-administration/deploy-on-kubernetes/greptimedb-operator-management), with compute/storage separation, native object storage (AWS S3, Azure Blob, etc.) and seamless cross-cloud access. |
| [Developer-Friendly](https://docs.greptime.com/user-guide/protocols/overview) | Access via SQL/PromQL interfaces, REST API, MySQL/PostgreSQL protocols, and popular ingestion [protocols](https://docs.greptime.com/user-guide/protocols/overview). |
| [Flexible Deployment](https://docs.greptime.com/user-guide/deployments/overview) | Deploy anywhere: edge (including ARM/[Android](https://docs.greptime.com/user-guide/deployments/run-on-android)) or cloud, with unified APIs and efficient data sync. |
| [Flexible Deployment](https://docs.greptime.com/user-guide/deployments-administration/overview) | Deploy anywhere: edge (including ARM/[Android](https://docs.greptime.com/user-guide/deployments-administration/run-on-android)) or cloud, with unified APIs and efficient data sync. |
Learn more in [Why GreptimeDB](https://docs.greptime.com/user-guide/concepts/why-greptimedb) and [Observability 2.0 and the Database for It](https://greptime.com/blogs/2025-04-25-greptimedb-observability2-new-database).

View File

@@ -598,3 +598,5 @@
| `logging.tracing_sample_ratio.default_ratio` | Float | `1.0` | -- |
| `tracing` | -- | -- | The tracing options. Only effect when compiled with `tokio-console` feature. |
| `tracing.tokio_console_addr` | String | Unset | The tokio console address. |
| `query` | -- | -- | -- |
| `query.parallelism` | Integer | `1` | Parallelism of the query engine for query sent by flownode.<br/>Default to 1, so it won't use too much cpu or memory |

View File

@@ -108,3 +108,8 @@ default_ratio = 1.0
## The tokio console address.
## @toml2docs:none-default
#+ tokio_console_addr = "127.0.0.1"
[query]
## Parallelism of the query engine for query sent by flownode.
## Default to 1, so it won't use too much cpu or memory
parallelism = 1

View File

@@ -48,4 +48,4 @@ Please refer to [SQL query](./query.sql) for GreptimeDB and Clickhouse, and [que
## Addition
- You can tune GreptimeDB's configuration to get better performance.
- You can setup GreptimeDB to use S3 as storage, see [here](https://docs.greptime.com/user-guide/deployments/configuration#storage-options).
- You can setup GreptimeDB to use S3 as storage, see [here](https://docs.greptime.com/user-guide/deployments-administration/configuration#storage-options).

View File

@@ -83,7 +83,7 @@ If you use the [Helm Chart](https://github.com/GreptimeTeam/helm-charts) to depl
- `monitoring.enabled=true`: Deploys a standalone GreptimeDB instance dedicated to monitoring the cluster;
- `grafana.enabled=true`: Deploys Grafana and automatically imports the monitoring dashboard;
The standalone GreptimeDB instance will collect metrics from your cluster, and the dashboard will be available in the Grafana UI. For detailed deployment instructions, please refer to our [Kubernetes deployment guide](https://docs.greptime.com/user-guide/deployments-administration/deploy-on-kubernetes/getting-started).
The standalone GreptimeDB instance will collect metrics from your cluster, and the dashboard will be available in the Grafana UI. For detailed deployment instructions, please refer to our [Kubernetes deployment guide](https://docs.greptime.com/user-guide/deployments-administration-administration/deploy-on-kubernetes/getting-started).
### Self-host Prometheus and import dashboards manually

View File

@@ -34,6 +34,7 @@ excludes = [
"src/sql/src/statements/drop/trigger.rs",
"src/sql/src/parsers/create_parser/trigger.rs",
"src/sql/src/parsers/show_parser/trigger.rs",
"src/mito2/src/extension.rs",
]
[properties]

View File

@@ -226,18 +226,20 @@ mod tests {
assert!(options.is_none());
let mut schema = ColumnSchema::new("test", ConcreteDataType::string_datatype(), true)
.with_fulltext_options(FulltextOptions {
enable: true,
analyzer: FulltextAnalyzer::English,
case_sensitive: false,
backend: FulltextBackend::Bloom,
})
.with_fulltext_options(FulltextOptions::new_unchecked(
true,
FulltextAnalyzer::English,
false,
FulltextBackend::Bloom,
10240,
0.01,
))
.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,\"backend\":\"bloom\"}"
"{\"enable\":true,\"analyzer\":\"English\",\"case-sensitive\":false,\"backend\":\"bloom\",\"granularity\":10240,\"false-positive-rate-in-10000\":100}"
);
assert_eq!(
options.options.get(INVERTED_INDEX_GRPC_KEY).unwrap(),
@@ -247,16 +249,18 @@ mod tests {
#[test]
fn test_options_with_fulltext() {
let fulltext = FulltextOptions {
enable: true,
analyzer: FulltextAnalyzer::English,
case_sensitive: false,
backend: FulltextBackend::Bloom,
};
let fulltext = FulltextOptions::new_unchecked(
true,
FulltextAnalyzer::English,
false,
FulltextBackend::Bloom,
10240,
0.01,
);
let options = options_from_fulltext(&fulltext).unwrap().unwrap();
assert_eq!(
options.options.get(FULLTEXT_GRPC_KEY).unwrap(),
"{\"enable\":true,\"analyzer\":\"English\",\"case-sensitive\":false,\"backend\":\"bloom\"}"
"{\"enable\":true,\"analyzer\":\"English\",\"case-sensitive\":false,\"backend\":\"bloom\",\"granularity\":10240,\"false-positive-rate-in-10000\":100}"
);
}

View File

@@ -5,6 +5,7 @@ edition.workspace = true
license.workspace = true
[features]
enterprise = []
testing = []
[lints]

View File

@@ -14,9 +14,11 @@
pub use client::{CachedKvBackend, CachedKvBackendBuilder, MetaKvBackend};
mod builder;
mod client;
mod manager;
mod table_cache;
pub use builder::KvBackendCatalogManagerBuilder;
pub use manager::KvBackendCatalogManager;
pub use table_cache::{new_table_cache, TableCache, TableCacheRef};

View File

@@ -0,0 +1,131 @@
// 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 common_catalog::consts::DEFAULT_CATALOG_NAME;
use common_meta::cache::LayeredCacheRegistryRef;
use common_meta::key::flow::FlowMetadataManager;
use common_meta::key::TableMetadataManager;
use common_meta::kv_backend::KvBackendRef;
use common_procedure::ProcedureManagerRef;
use moka::sync::Cache;
use partition::manager::PartitionRuleManager;
#[cfg(feature = "enterprise")]
use crate::information_schema::InformationSchemaTableFactoryRef;
use crate::information_schema::{InformationExtensionRef, InformationSchemaProvider};
use crate::kvbackend::manager::{SystemCatalog, CATALOG_CACHE_MAX_CAPACITY};
use crate::kvbackend::KvBackendCatalogManager;
use crate::process_manager::ProcessManagerRef;
use crate::system_schema::pg_catalog::PGCatalogProvider;
pub struct KvBackendCatalogManagerBuilder {
information_extension: InformationExtensionRef,
backend: KvBackendRef,
cache_registry: LayeredCacheRegistryRef,
procedure_manager: Option<ProcedureManagerRef>,
process_manager: Option<ProcessManagerRef>,
#[cfg(feature = "enterprise")]
extra_information_table_factories:
std::collections::HashMap<String, InformationSchemaTableFactoryRef>,
}
impl KvBackendCatalogManagerBuilder {
pub fn new(
information_extension: InformationExtensionRef,
backend: KvBackendRef,
cache_registry: LayeredCacheRegistryRef,
) -> Self {
Self {
information_extension,
backend,
cache_registry,
procedure_manager: None,
process_manager: None,
#[cfg(feature = "enterprise")]
extra_information_table_factories: std::collections::HashMap::new(),
}
}
pub fn with_procedure_manager(mut self, procedure_manager: ProcedureManagerRef) -> Self {
self.procedure_manager = Some(procedure_manager);
self
}
pub fn with_process_manager(mut self, process_manager: ProcessManagerRef) -> Self {
self.process_manager = Some(process_manager);
self
}
/// Sets the extra information tables.
#[cfg(feature = "enterprise")]
pub fn with_extra_information_table_factories(
mut self,
factories: std::collections::HashMap<String, InformationSchemaTableFactoryRef>,
) -> Self {
self.extra_information_table_factories = factories;
self
}
pub fn build(self) -> Arc<KvBackendCatalogManager> {
let Self {
information_extension,
backend,
cache_registry,
procedure_manager,
process_manager,
#[cfg(feature = "enterprise")]
extra_information_table_factories,
} = self;
Arc::new_cyclic(|me| KvBackendCatalogManager {
information_extension,
partition_manager: Arc::new(PartitionRuleManager::new(
backend.clone(),
cache_registry
.get()
.expect("Failed to get table_route_cache"),
)),
table_metadata_manager: Arc::new(TableMetadataManager::new(backend.clone())),
system_catalog: SystemCatalog {
catalog_manager: me.clone(),
catalog_cache: Cache::new(CATALOG_CACHE_MAX_CAPACITY),
pg_catalog_cache: Cache::new(CATALOG_CACHE_MAX_CAPACITY),
information_schema_provider: {
let provider = InformationSchemaProvider::new(
DEFAULT_CATALOG_NAME.to_string(),
me.clone(),
Arc::new(FlowMetadataManager::new(backend.clone())),
process_manager.clone(),
backend.clone(),
);
#[cfg(feature = "enterprise")]
let provider = provider
.with_extra_table_factories(extra_information_table_factories.clone());
Arc::new(provider)
},
pg_catalog_provider: Arc::new(PGCatalogProvider::new(
DEFAULT_CATALOG_NAME.to_string(),
me.clone(),
)),
backend,
process_manager,
#[cfg(feature = "enterprise")]
extra_information_table_factories,
},
cache_registry,
procedure_manager,
})
}
}

View File

@@ -13,7 +13,7 @@
// limitations under the License.
use std::any::Any;
use std::collections::{BTreeSet, HashSet};
use std::collections::BTreeSet;
use std::sync::{Arc, Weak};
use async_stream::try_stream;
@@ -30,13 +30,13 @@ use common_meta::key::flow::FlowMetadataManager;
use common_meta::key::schema_name::SchemaNameKey;
use common_meta::key::table_info::{TableInfoManager, TableInfoValue};
use common_meta::key::table_name::TableNameKey;
use common_meta::key::{TableMetadataManager, TableMetadataManagerRef};
use common_meta::key::TableMetadataManagerRef;
use common_meta::kv_backend::KvBackendRef;
use common_procedure::ProcedureManagerRef;
use futures_util::stream::BoxStream;
use futures_util::{StreamExt, TryStreamExt};
use moka::sync::Cache;
use partition::manager::{PartitionRuleManager, PartitionRuleManagerRef};
use partition::manager::PartitionRuleManagerRef;
use session::context::{Channel, QueryContext};
use snafu::prelude::*;
use store_api::metric_engine_consts::METRIC_ENGINE_NAME;
@@ -52,6 +52,8 @@ use crate::error::{
CacheNotFoundSnafu, GetTableCacheSnafu, InvalidTableInfoInCatalogSnafu, ListCatalogsSnafu,
ListSchemasSnafu, ListTablesSnafu, Result, TableMetadataManagerSnafu,
};
#[cfg(feature = "enterprise")]
use crate::information_schema::InformationSchemaTableFactoryRef;
use crate::information_schema::{InformationExtensionRef, InformationSchemaProvider};
use crate::kvbackend::TableCacheRef;
use crate::process_manager::ProcessManagerRef;
@@ -67,60 +69,22 @@ use crate::CatalogManager;
#[derive(Clone)]
pub struct KvBackendCatalogManager {
/// Provides the extension methods for the `information_schema` tables
information_extension: InformationExtensionRef,
pub(super) information_extension: InformationExtensionRef,
/// Manages partition rules.
partition_manager: PartitionRuleManagerRef,
pub(super) partition_manager: PartitionRuleManagerRef,
/// Manages table metadata.
table_metadata_manager: TableMetadataManagerRef,
pub(super) table_metadata_manager: TableMetadataManagerRef,
/// A sub-CatalogManager that handles system tables
system_catalog: SystemCatalog,
pub(super) system_catalog: SystemCatalog,
/// Cache registry for all caches.
cache_registry: LayeredCacheRegistryRef,
pub(super) cache_registry: LayeredCacheRegistryRef,
/// Only available in `Standalone` mode.
procedure_manager: Option<ProcedureManagerRef>,
pub(super) procedure_manager: Option<ProcedureManagerRef>,
}
const CATALOG_CACHE_MAX_CAPACITY: u64 = 128;
pub(super) const CATALOG_CACHE_MAX_CAPACITY: u64 = 128;
impl KvBackendCatalogManager {
pub fn new(
information_extension: InformationExtensionRef,
backend: KvBackendRef,
cache_registry: LayeredCacheRegistryRef,
procedure_manager: Option<ProcedureManagerRef>,
process_manager: Option<ProcessManagerRef>,
) -> Arc<Self> {
Arc::new_cyclic(|me| Self {
information_extension,
partition_manager: Arc::new(PartitionRuleManager::new(
backend.clone(),
cache_registry
.get()
.expect("Failed to get table_route_cache"),
)),
table_metadata_manager: Arc::new(TableMetadataManager::new(backend.clone())),
system_catalog: SystemCatalog {
catalog_manager: me.clone(),
catalog_cache: Cache::new(CATALOG_CACHE_MAX_CAPACITY),
pg_catalog_cache: Cache::new(CATALOG_CACHE_MAX_CAPACITY),
information_schema_provider: Arc::new(InformationSchemaProvider::new(
DEFAULT_CATALOG_NAME.to_string(),
me.clone(),
Arc::new(FlowMetadataManager::new(backend.clone())),
process_manager.clone(),
)),
pg_catalog_provider: Arc::new(PGCatalogProvider::new(
DEFAULT_CATALOG_NAME.to_string(),
me.clone(),
)),
backend,
process_manager,
},
cache_registry,
procedure_manager,
})
}
pub fn view_info_cache(&self) -> Result<ViewInfoCacheRef> {
self.cache_registry.get().context(CacheNotFoundSnafu {
name: "view_info_cache",
@@ -166,32 +130,29 @@ impl KvBackendCatalogManager {
.context(TableMetadataManagerSnafu)?
{
let mut new_table_info = (*table.table_info()).clone();
// Gather all column names from the logical table
let logical_column_names: HashSet<_> = new_table_info
.meta
.schema
.column_schemas()
.iter()
.map(|col| &col.name)
.collect();
// Only preserve partition key indices where the corresponding columns exist in logical table
// Remap partition key indices from physical table to logical table
new_table_info.meta.partition_key_indices = physical_table_info_value
.table_info
.meta
.partition_key_indices
.iter()
.filter(|&&index| {
.filter_map(|&physical_index| {
// Get the column name from the physical table using the physical index
physical_table_info_value
.table_info
.meta
.schema
.column_schemas
.get(index)
.map(|physical_column| logical_column_names.contains(&physical_column.name))
.unwrap_or(false)
.get(physical_index)
.and_then(|physical_column| {
// Find the corresponding index in the logical table schema
new_table_info
.meta
.schema
.column_index_by_name(physical_column.name.as_str())
})
})
.cloned()
.collect();
let new_table = DistTable::table(Arc::new(new_table_info));
@@ -510,16 +471,19 @@ fn build_table(table_info_value: TableInfoValue) -> Result<TableRef> {
/// - information_schema.{tables}
/// - pg_catalog.{tables}
#[derive(Clone)]
struct SystemCatalog {
catalog_manager: Weak<KvBackendCatalogManager>,
catalog_cache: Cache<String, Arc<InformationSchemaProvider>>,
pg_catalog_cache: Cache<String, Arc<PGCatalogProvider>>,
pub(super) struct SystemCatalog {
pub(super) catalog_manager: Weak<KvBackendCatalogManager>,
pub(super) catalog_cache: Cache<String, Arc<InformationSchemaProvider>>,
pub(super) pg_catalog_cache: Cache<String, Arc<PGCatalogProvider>>,
// system_schema_provider for default catalog
information_schema_provider: Arc<InformationSchemaProvider>,
pg_catalog_provider: Arc<PGCatalogProvider>,
backend: KvBackendRef,
process_manager: Option<ProcessManagerRef>,
pub(super) information_schema_provider: Arc<InformationSchemaProvider>,
pub(super) pg_catalog_provider: Arc<PGCatalogProvider>,
pub(super) backend: KvBackendRef,
pub(super) process_manager: Option<ProcessManagerRef>,
#[cfg(feature = "enterprise")]
pub(super) extra_information_table_factories:
std::collections::HashMap<String, InformationSchemaTableFactoryRef>,
}
impl SystemCatalog {
@@ -583,12 +547,17 @@ impl SystemCatalog {
if schema == INFORMATION_SCHEMA_NAME {
let information_schema_provider =
self.catalog_cache.get_with_by_ref(catalog, move || {
Arc::new(InformationSchemaProvider::new(
let provider = InformationSchemaProvider::new(
catalog.to_string(),
self.catalog_manager.clone(),
Arc::new(FlowMetadataManager::new(self.backend.clone())),
self.process_manager.clone(),
))
self.backend.clone(),
);
#[cfg(feature = "enterprise")]
let provider = provider
.with_extra_table_factories(self.extra_information_table_factories.clone());
Arc::new(provider)
});
information_schema_provider.table(table_name)
} else if schema == PG_CATALOG_NAME && channel == Channel::Postgres {

View File

@@ -352,11 +352,13 @@ impl MemoryCatalogManager {
}
fn create_catalog_entry(self: &Arc<Self>, catalog: String) -> SchemaEntries {
let backend = Arc::new(MemoryKvBackend::new());
let information_schema_provider = InformationSchemaProvider::new(
catalog,
Arc::downgrade(self) as Weak<dyn CatalogManager>,
Arc::new(FlowMetadataManager::new(Arc::new(MemoryKvBackend::new()))),
Arc::new(FlowMetadataManager::new(backend.clone())),
None, // we don't need ProcessManager on regions server.
backend,
);
let information_schema = information_schema_provider.tables().clone();

View File

@@ -15,7 +15,7 @@
pub mod information_schema;
mod memory_table;
pub mod pg_catalog;
mod predicate;
pub mod predicate;
mod utils;
use std::collections::HashMap;
@@ -96,7 +96,7 @@ trait SystemSchemaProviderInner {
}
}
pub(crate) trait SystemTable {
pub trait SystemTable {
fn table_id(&self) -> TableId;
fn table_name(&self) -> &'static str;
@@ -110,7 +110,7 @@ pub(crate) trait SystemTable {
}
}
pub(crate) type SystemTableRef = Arc<dyn SystemTable + Send + Sync>;
pub type SystemTableRef = Arc<dyn SystemTable + Send + Sync>;
struct SystemTableDataSource {
table: SystemTableRef,

View File

@@ -38,6 +38,7 @@ use common_meta::cluster::NodeInfo;
use common_meta::datanode::RegionStat;
use common_meta::key::flow::flow_state::FlowStat;
use common_meta::key::flow::FlowMetadataManager;
use common_meta::kv_backend::KvBackendRef;
use common_procedure::ProcedureInfo;
use common_recordbatch::SendableRecordBatchStream;
use datatypes::schema::SchemaRef;
@@ -112,6 +113,25 @@ macro_rules! setup_memory_table {
};
}
#[cfg(feature = "enterprise")]
pub struct MakeInformationTableRequest {
pub catalog_name: String,
pub catalog_manager: Weak<dyn CatalogManager>,
pub kv_backend: KvBackendRef,
}
/// A factory trait for making information schema tables.
///
/// This trait allows for extensibility of the information schema by providing
/// a way to dynamically create custom information schema tables.
#[cfg(feature = "enterprise")]
pub trait InformationSchemaTableFactory {
fn make_information_table(&self, req: MakeInformationTableRequest) -> SystemTableRef;
}
#[cfg(feature = "enterprise")]
pub type InformationSchemaTableFactoryRef = Arc<dyn InformationSchemaTableFactory + Send + Sync>;
/// The `information_schema` tables info provider.
pub struct InformationSchemaProvider {
catalog_name: String,
@@ -119,6 +139,10 @@ pub struct InformationSchemaProvider {
process_manager: Option<ProcessManagerRef>,
flow_metadata_manager: Arc<FlowMetadataManager>,
tables: HashMap<String, TableRef>,
#[allow(dead_code)]
kv_backend: KvBackendRef,
#[cfg(feature = "enterprise")]
extra_table_factories: HashMap<String, InformationSchemaTableFactoryRef>,
}
impl SystemSchemaProvider for InformationSchemaProvider {
@@ -128,6 +152,7 @@ impl SystemSchemaProvider for InformationSchemaProvider {
&self.tables
}
}
impl SystemSchemaProviderInner for InformationSchemaProvider {
fn catalog_name(&self) -> &str {
&self.catalog_name
@@ -215,7 +240,22 @@ impl SystemSchemaProviderInner for InformationSchemaProvider {
.process_manager
.as_ref()
.map(|p| Arc::new(InformationSchemaProcessList::new(p.clone())) as _),
_ => None,
table_name => {
#[cfg(feature = "enterprise")]
return self.extra_table_factories.get(table_name).map(|factory| {
let req = MakeInformationTableRequest {
catalog_name: self.catalog_name.clone(),
catalog_manager: self.catalog_manager.clone(),
kv_backend: self.kv_backend.clone(),
};
factory.make_information_table(req)
});
#[cfg(not(feature = "enterprise"))]
{
let _ = table_name;
None
}
}
}
}
}
@@ -226,6 +266,7 @@ impl InformationSchemaProvider {
catalog_manager: Weak<dyn CatalogManager>,
flow_metadata_manager: Arc<FlowMetadataManager>,
process_manager: Option<ProcessManagerRef>,
kv_backend: KvBackendRef,
) -> Self {
let mut provider = Self {
catalog_name,
@@ -233,6 +274,9 @@ impl InformationSchemaProvider {
flow_metadata_manager,
process_manager,
tables: HashMap::new(),
kv_backend,
#[cfg(feature = "enterprise")]
extra_table_factories: HashMap::new(),
};
provider.build_tables();
@@ -240,6 +284,16 @@ impl InformationSchemaProvider {
provider
}
#[cfg(feature = "enterprise")]
pub(crate) fn with_extra_table_factories(
mut self,
factories: HashMap<String, InformationSchemaTableFactoryRef>,
) -> Self {
self.extra_table_factories = factories;
self.build_tables();
self
}
fn build_tables(&mut self) {
let mut tables = HashMap::new();
@@ -290,16 +344,19 @@ impl InformationSchemaProvider {
if let Some(process_list) = self.build_table(PROCESS_LIST) {
tables.insert(PROCESS_LIST.to_string(), process_list);
}
#[cfg(feature = "enterprise")]
for name in self.extra_table_factories.keys() {
tables.insert(name.to_string(), self.build_table(name).expect(name));
}
// Add memory tables
for name in MEMORY_TABLES.iter() {
tables.insert((*name).to_string(), self.build_table(name).expect(name));
}
self.tables = tables;
}
}
trait InformationTable {
pub trait InformationTable {
fn table_id(&self) -> TableId;
fn table_name(&self) -> &'static str;

View File

@@ -48,3 +48,4 @@ pub const FLOWS: &str = "flows";
pub const PROCEDURE_INFO: &str = "procedure_info";
pub const REGION_STATISTICS: &str = "region_statistics";
pub const PROCESS_LIST: &str = "process_list";
pub const TRIGGER_LIST: &str = "trigger_list";

View File

@@ -207,6 +207,7 @@ mod tests {
use session::context::QueryContext;
use super::*;
use crate::kvbackend::KvBackendCatalogManagerBuilder;
use crate::memory::MemoryCatalogManager;
#[test]
@@ -323,13 +324,13 @@ mod tests {
.build(),
);
let catalog_manager = KvBackendCatalogManager::new(
let catalog_manager = KvBackendCatalogManagerBuilder::new(
Arc::new(NoopInformationExtension),
backend.clone(),
layered_cache_registry,
None,
None,
);
)
.build();
let table_metadata_manager = TableMetadataManager::new(backend);
let mut view_info = common_meta::key::test_utils::new_test_table_info(1024, vec![]);
view_info.table_type = TableType::View;

View File

@@ -43,7 +43,6 @@ common-time.workspace = true
common-version.workspace = true
common-wal.workspace = true
datatypes.workspace = true
either = "1.8"
etcd-client.workspace = true
futures.workspace = true
humantime.workspace = true

View File

@@ -160,6 +160,7 @@ fn create_table_info(table_id: TableId, table_name: TableName) -> RawTableInfo {
options: Default::default(),
region_numbers: (1..=100).collect(),
partition_key_indices: vec![],
column_ids: vec![],
};
RawTableInfo {

View File

@@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
#[allow(clippy::print_stdout)]
#![allow(clippy::print_stdout)]
mod bench;
mod data;
mod database;

View File

@@ -241,7 +241,6 @@ impl RepairTool {
let alter_table_request = alter_table::make_alter_region_request_for_peer(
logical_table_id,
&alter_table_expr,
full_table_metadata.table_info.ident.version,
peer,
physical_region_routes,
)?;

View File

@@ -66,7 +66,6 @@ pub fn generate_alter_table_expr_for_all_columns(
pub fn make_alter_region_request_for_peer(
logical_table_id: TableId,
alter_table_expr: &AlterTableExpr,
schema_version: u64,
peer: &Peer,
region_routes: &[RegionRoute],
) -> Result<RegionRequest> {
@@ -74,7 +73,7 @@ pub fn make_alter_region_request_for_peer(
let mut requests = Vec::with_capacity(regions_on_this_peer.len());
for region_number in &regions_on_this_peer {
let region_id = RegionId::new(logical_table_id, *region_number);
let request = make_alter_region_request(region_id, alter_table_expr, schema_version);
let request = make_alter_region_request(region_id, alter_table_expr);
requests.push(request);
}

View File

@@ -31,7 +31,7 @@ use base64::prelude::BASE64_STANDARD;
use base64::Engine;
use common_catalog::build_db_string;
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
use common_error::ext::{BoxedError, ErrorExt};
use common_error::ext::BoxedError;
use common_grpc::flight::do_put::DoPutResponse;
use common_grpc::flight::{FlightDecoder, FlightMessage};
use common_query::Output;
@@ -48,7 +48,7 @@ use tonic::transport::Channel;
use crate::error::{
ConvertFlightDataSnafu, Error, FlightGetSnafu, IllegalFlightMessagesSnafu,
InvalidTonicMetadataValueSnafu, ServerSnafu,
InvalidTonicMetadataValueSnafu,
};
use crate::{error, from_grpc_response, Client, Result};
@@ -196,12 +196,22 @@ impl Database {
/// Retry if connection fails, max_retries is the max number of retries, so the total wait time
/// is `max_retries * GRPC_CONN_TIMEOUT`
pub async fn handle_with_retry(&self, request: Request, max_retries: u32) -> Result<u32> {
pub async fn handle_with_retry(
&self,
request: Request,
max_retries: u32,
hints: &[(&str, &str)],
) -> Result<u32> {
let mut client = make_database_client(&self.client)?.inner;
let mut retries = 0;
let request = self.to_rpc_request(request);
loop {
let raw_response = client.handle(request.clone()).await;
let mut tonic_request = tonic::Request::new(request.clone());
let metadata = tonic_request.metadata_mut();
Self::put_hints(metadata, hints)?;
let raw_response = client.handle(tonic_request).await;
match (raw_response, retries < max_retries) {
(Ok(resp), _) => return from_grpc_response(resp.into_inner()),
(Err(err), true) => {
@@ -292,21 +302,16 @@ impl Database {
let response = client.mut_inner().do_get(request).await.or_else(|e| {
let tonic_code = e.code();
let e: Error = e.into();
let code = e.status_code();
let msg = e.to_string();
let error =
Err(BoxedError::new(ServerSnafu { code, msg }.build())).with_context(|_| {
FlightGetSnafu {
addr: client.addr().to_string(),
tonic_code,
}
});
error!(
"Failed to do Flight get, addr: {}, code: {}, source: {:?}",
client.addr(),
tonic_code,
error
e
);
let error = Err(BoxedError::new(e)).with_context(|_| FlightGetSnafu {
addr: client.addr().to_string(),
tonic_code,
});
error
})?;
@@ -436,8 +441,11 @@ mod tests {
use api::v1::auth_header::AuthScheme;
use api::v1::{AuthHeader, Basic};
use common_error::status_code::StatusCode;
use tonic::{Code, Status};
use super::*;
use crate::error::TonicSnafu;
#[test]
fn test_flight_ctx() {
@@ -460,4 +468,19 @@ mod tests {
})
)
}
#[test]
fn test_from_tonic_status() {
let expected = TonicSnafu {
code: StatusCode::Internal,
msg: "blabla".to_string(),
tonic_code: Code::Internal,
}
.build();
let status = Status::new(Code::Internal, "blabla");
let actual: Error = status.into();
assert_eq!(expected.to_string(), actual.to_string());
}
}

View File

@@ -14,13 +14,13 @@
use std::any::Any;
use common_error::define_from_tonic_status;
use common_error::ext::{BoxedError, ErrorExt};
use common_error::status_code::{convert_tonic_code_to_status_code, StatusCode};
use common_error::{GREPTIME_DB_HEADER_ERROR_CODE, GREPTIME_DB_HEADER_ERROR_MSG};
use common_error::status_code::StatusCode;
use common_macro::stack_trace_debug;
use snafu::{location, Location, Snafu};
use tonic::metadata::errors::InvalidMetadataValue;
use tonic::{Code, Status};
use tonic::Code;
#[derive(Snafu)]
#[snafu(visibility(pub))]
@@ -124,6 +124,15 @@ pub enum Error {
location: Location,
source: datatypes::error::Error,
},
#[snafu(display("{}", msg))]
Tonic {
code: StatusCode,
msg: String,
tonic_code: Code,
#[snafu(implicit)]
location: Location,
},
}
pub type Result<T> = std::result::Result<T, Error>;
@@ -135,7 +144,7 @@ impl ErrorExt for Error {
| Error::MissingField { .. }
| Error::IllegalDatabaseResponse { .. } => StatusCode::Internal,
Error::Server { code, .. } => *code,
Error::Server { code, .. } | Error::Tonic { code, .. } => *code,
Error::FlightGet { source, .. }
| Error::RegionServer { source, .. }
| Error::FlowServer { source, .. } => source.status_code(),
@@ -153,34 +162,7 @@ impl ErrorExt for Error {
}
}
impl From<Status> for Error {
fn from(e: Status) -> Self {
fn get_metadata_value(e: &Status, key: &str) -> Option<String> {
e.metadata()
.get(key)
.and_then(|v| String::from_utf8(v.as_bytes().to_vec()).ok())
}
let code = get_metadata_value(&e, GREPTIME_DB_HEADER_ERROR_CODE).and_then(|s| {
if let Ok(code) = s.parse::<u32>() {
StatusCode::from_u32(code)
} else {
None
}
});
let tonic_code = e.code();
let code = code.unwrap_or_else(|| convert_tonic_code_to_status_code(tonic_code));
let msg = get_metadata_value(&e, GREPTIME_DB_HEADER_ERROR_MSG)
.unwrap_or_else(|| e.message().to_string());
Self::Server {
code,
msg,
location: location!(),
}
}
}
define_from_tonic_status!(Error, Tonic);
impl Error {
pub fn should_retry(&self) -> bool {

View File

@@ -21,7 +21,7 @@ use arc_swap::ArcSwapOption;
use arrow_flight::Ticket;
use async_stream::stream;
use async_trait::async_trait;
use common_error::ext::{BoxedError, ErrorExt};
use common_error::ext::BoxedError;
use common_error::status_code::StatusCode;
use common_grpc::flight::{FlightDecoder, FlightMessage};
use common_meta::error::{self as meta_error, Result as MetaResult};
@@ -107,24 +107,18 @@ impl RegionRequester {
.mut_inner()
.do_get(ticket)
.await
.map_err(|e| {
.or_else(|e| {
let tonic_code = e.code();
let e: error::Error = e.into();
let code = e.status_code();
let msg = e.to_string();
let error = ServerSnafu { code, msg }
.fail::<()>()
.map_err(BoxedError::new)
.with_context(|_| FlightGetSnafu {
tonic_code,
addr: flight_client.addr().to_string(),
})
.unwrap_err();
error!(
e; "Failed to do Flight get, addr: {}, code: {}",
flight_client.addr(),
tonic_code
);
let error = Err(BoxedError::new(e)).with_context(|_| FlightGetSnafu {
addr: flight_client.addr().to_string(),
tonic_code,
});
error
})?;

View File

@@ -16,7 +16,7 @@ default = [
"meta-srv/pg_kvbackend",
"meta-srv/mysql_kvbackend",
]
enterprise = ["common-meta/enterprise", "frontend/enterprise", "meta-srv/enterprise"]
enterprise = ["common-meta/enterprise", "frontend/enterprise", "meta-srv/enterprise", "catalog/enterprise"]
tokio-console = ["common-telemetry/tokio-console"]
[lints]
@@ -52,7 +52,6 @@ common-version.workspace = true
common-wal.workspace = true
datanode.workspace = true
datatypes.workspace = true
either = "1.8"
etcd-client.workspace = true
file-engine.workspace = true
flow.workspace = true

View File

@@ -20,11 +20,11 @@ use cmd::error::{InitTlsProviderSnafu, Result};
use cmd::options::GlobalOptions;
use cmd::{cli, datanode, flownode, frontend, metasrv, standalone, App};
use common_base::Plugins;
use common_version::version;
use common_version::{verbose_version, version};
use servers::install_ring_crypto_provider;
#[derive(Parser)]
#[command(name = "greptime", author, version, long_version = version(), about)]
#[command(name = "greptime", author, version, long_version = verbose_version(), about)]
#[command(propagate_version = true)]
pub(crate) struct Command {
#[clap(subcommand)]
@@ -143,10 +143,8 @@ async fn start(cli: Command) -> Result<()> {
}
fn setup_human_panic() {
human_panic::setup_panic!(
human_panic::Metadata::new("GreptimeDB", env!("CARGO_PKG_VERSION"))
.homepage("https://github.com/GreptimeTeam/greptimedb/discussions")
);
human_panic::setup_panic!(human_panic::Metadata::new("GreptimeDB", version())
.homepage("https://github.com/GreptimeTeam/greptimedb/discussions"));
common_telemetry::set_panic_hook();
}

View File

@@ -19,7 +19,7 @@ use catalog::kvbackend::MetaKvBackend;
use common_base::Plugins;
use common_meta::cache::LayeredCacheRegistryBuilder;
use common_telemetry::info;
use common_version::{short_version, version};
use common_version::{short_version, verbose_version};
use datanode::datanode::DatanodeBuilder;
use datanode::service::DatanodeServiceBuilder;
use meta_client::MetaClientType;
@@ -67,7 +67,7 @@ impl InstanceBuilder {
None,
);
log_versions(version(), short_version(), APP_NAME);
log_versions(verbose_version(), short_version(), APP_NAME);
create_resource_limit_metrics(APP_NAME);
plugins::setup_datanode_plugins(plugins, &opts.plugins, dn_opts)

View File

@@ -18,7 +18,7 @@ use std::time::Duration;
use cache::{build_fundamental_cache_registry, with_default_composite_cache_registry};
use catalog::information_extension::DistributedInformationExtension;
use catalog::kvbackend::{CachedKvBackendBuilder, KvBackendCatalogManager, MetaKvBackend};
use catalog::kvbackend::{CachedKvBackendBuilder, KvBackendCatalogManagerBuilder, MetaKvBackend};
use clap::Parser;
use client::client_manager::NodeClients;
use common_base::Plugins;
@@ -32,7 +32,7 @@ use common_meta::key::flow::FlowMetadataManager;
use common_meta::key::TableMetadataManager;
use common_telemetry::info;
use common_telemetry::logging::{TracingOptions, DEFAULT_LOGGING_DIR};
use common_version::{short_version, version};
use common_version::{short_version, verbose_version};
use flow::{
get_flow_auth_options, FlownodeBuilder, FlownodeInstance, FlownodeServiceBuilder,
FrontendClient, FrontendInvoker,
@@ -279,7 +279,7 @@ impl StartCommand {
None,
);
log_versions(version(), short_version(), APP_NAME);
log_versions(verbose_version(), short_version(), APP_NAME);
create_resource_limit_metrics(APP_NAME);
info!("Flownode start command: {:#?}", self);
@@ -342,13 +342,12 @@ impl StartCommand {
let information_extension =
Arc::new(DistributedInformationExtension::new(meta_client.clone()));
let catalog_manager = KvBackendCatalogManager::new(
let catalog_manager = KvBackendCatalogManagerBuilder::new(
information_extension,
cached_meta_backend.clone(),
layered_cache_registry.clone(),
None,
None,
);
)
.build();
let table_metadata_manager =
Arc::new(TableMetadataManager::new(cached_meta_backend.clone()));
@@ -371,8 +370,11 @@ impl StartCommand {
let flow_metadata_manager = Arc::new(FlowMetadataManager::new(cached_meta_backend.clone()));
let flow_auth_header = get_flow_auth_options(&opts).context(StartFlownodeSnafu)?;
let frontend_client =
FrontendClient::from_meta_client(meta_client.clone(), flow_auth_header);
let frontend_client = FrontendClient::from_meta_client(
meta_client.clone(),
flow_auth_header,
opts.query.clone(),
);
let frontend_client = Arc::new(frontend_client);
let flownode_builder = FlownodeBuilder::new(
opts.clone(),

View File

@@ -19,7 +19,7 @@ use std::time::Duration;
use async_trait::async_trait;
use cache::{build_fundamental_cache_registry, with_default_composite_cache_registry};
use catalog::information_extension::DistributedInformationExtension;
use catalog::kvbackend::{CachedKvBackendBuilder, KvBackendCatalogManager, MetaKvBackend};
use catalog::kvbackend::{CachedKvBackendBuilder, KvBackendCatalogManagerBuilder, MetaKvBackend};
use catalog::process_manager::ProcessManager;
use clap::Parser;
use client::client_manager::NodeClients;
@@ -33,7 +33,7 @@ use common_meta::heartbeat::handler::HandlerGroupExecutor;
use common_telemetry::info;
use common_telemetry::logging::{TracingOptions, DEFAULT_LOGGING_DIR};
use common_time::timezone::set_default_timezone;
use common_version::{short_version, version};
use common_version::{short_version, verbose_version};
use frontend::frontend::Frontend;
use frontend::heartbeat::HeartbeatTask;
use frontend::instance::builder::FrontendBuilder;
@@ -102,7 +102,7 @@ impl App for Instance {
#[derive(Parser)]
pub struct Command {
#[clap(subcommand)]
subcmd: SubCommand,
pub subcmd: SubCommand,
}
impl Command {
@@ -116,7 +116,7 @@ impl Command {
}
#[derive(Parser)]
enum SubCommand {
pub enum SubCommand {
Start(StartCommand),
}
@@ -153,7 +153,7 @@ pub struct StartCommand {
#[clap(long)]
postgres_addr: Option<String>,
#[clap(short, long)]
config_file: Option<String>,
pub config_file: Option<String>,
#[clap(short, long)]
influxdb_enable: Option<bool>,
#[clap(long, value_delimiter = ',', num_args = 1..)]
@@ -169,7 +169,7 @@ pub struct StartCommand {
#[clap(long)]
disable_dashboard: Option<bool>,
#[clap(long, default_value = "GREPTIMEDB_FRONTEND")]
env_prefix: String,
pub env_prefix: String,
}
impl StartCommand {
@@ -282,7 +282,7 @@ impl StartCommand {
opts.component.slow_query.as_ref(),
);
log_versions(version(), short_version(), APP_NAME);
log_versions(verbose_version(), short_version(), APP_NAME);
create_resource_limit_metrics(APP_NAME);
info!("Frontend start command: {:#?}", self);
@@ -350,13 +350,20 @@ impl StartCommand {
addrs::resolve_addr(&opts.grpc.bind_addr, Some(&opts.grpc.server_addr)),
Some(meta_client.clone()),
));
let catalog_manager = KvBackendCatalogManager::new(
let builder = KvBackendCatalogManagerBuilder::new(
information_extension,
cached_meta_backend.clone(),
layered_cache_registry.clone(),
None,
Some(process_manager.clone()),
);
)
.with_process_manager(process_manager.clone());
#[cfg(feature = "enterprise")]
let builder = if let Some(factories) = plugins.get() {
builder.with_extra_information_table_factories(factories)
} else {
builder
};
let catalog_manager = builder.build();
let executor = HandlerGroupExecutor::new(vec![
Arc::new(ParseMailboxMessageHandler),

View File

@@ -112,7 +112,7 @@ pub trait App: Send {
pub fn log_versions(version: &str, short_version: &str, app: &str) {
// Report app version as gauge.
APP_VERSION
.with_label_values(&[env!("CARGO_PKG_VERSION"), short_version, app])
.with_label_values(&[common_version::version(), short_version, app])
.inc();
// Log version and argument flags.

View File

@@ -22,7 +22,7 @@ use common_base::Plugins;
use common_config::Configurable;
use common_telemetry::info;
use common_telemetry::logging::{TracingOptions, DEFAULT_LOGGING_DIR};
use common_version::{short_version, version};
use common_version::{short_version, verbose_version};
use meta_srv::bootstrap::MetasrvInstance;
use meta_srv::metasrv::BackendImpl;
use snafu::ResultExt;
@@ -54,6 +54,10 @@ impl Instance {
pub fn get_inner(&self) -> &MetasrvInstance {
&self.instance
}
pub fn mut_inner(&mut self) -> &mut MetasrvInstance {
&mut self.instance
}
}
#[async_trait]
@@ -320,7 +324,7 @@ impl StartCommand {
None,
);
log_versions(version(), short_version(), APP_NAME);
log_versions(verbose_version(), short_version(), APP_NAME);
create_resource_limit_metrics(APP_NAME);
info!("Metasrv start command: {:#?}", self);
@@ -336,12 +340,12 @@ impl StartCommand {
.await
.context(StartMetaServerSnafu)?;
let builder = meta_srv::bootstrap::metasrv_builder(&opts, plugins.clone(), None)
let builder = meta_srv::bootstrap::metasrv_builder(&opts, plugins, None)
.await
.context(error::BuildMetaServerSnafu)?;
let metasrv = builder.build().await.context(error::BuildMetaServerSnafu)?;
let instance = MetasrvInstance::new(opts, plugins, metasrv)
let instance = MetasrvInstance::new(metasrv)
.await
.context(error::BuildMetaServerSnafu)?;

View File

@@ -20,7 +20,7 @@ use std::{fs, path};
use async_trait::async_trait;
use cache::{build_fundamental_cache_registry, with_default_composite_cache_registry};
use catalog::information_schema::InformationExtension;
use catalog::kvbackend::KvBackendCatalogManager;
use catalog::kvbackend::KvBackendCatalogManagerBuilder;
use catalog::process_manager::ProcessManager;
use clap::Parser;
use client::api::v1::meta::RegionRole;
@@ -51,7 +51,7 @@ use common_telemetry::logging::{
LoggingOptions, SlowQueryOptions, TracingOptions, DEFAULT_LOGGING_DIR,
};
use common_time::timezone::set_default_timezone;
use common_version::{short_version, version};
use common_version::{short_version, verbose_version};
use common_wal::config::DatanodeWalConfig;
use datanode::config::{DatanodeOptions, ProcedureConfig, RegionEngineConfig, StorageConfig};
use datanode::datanode::{Datanode, DatanodeBuilder};
@@ -485,7 +485,7 @@ impl StartCommand {
opts.component.slow_query.as_ref(),
);
log_versions(version(), short_version(), APP_NAME);
log_versions(verbose_version(), short_version(), APP_NAME);
create_resource_limit_metrics(APP_NAME);
info!("Standalone start command: {:#?}", self);
@@ -544,13 +544,20 @@ impl StartCommand {
));
let process_manager = Arc::new(ProcessManager::new(opts.grpc.server_addr.clone(), None));
let catalog_manager = KvBackendCatalogManager::new(
let builder = KvBackendCatalogManagerBuilder::new(
information_extension.clone(),
kv_backend.clone(),
layered_cache_registry.clone(),
Some(procedure_manager.clone()),
Some(process_manager.clone()),
);
)
.with_procedure_manager(procedure_manager.clone())
.with_process_manager(process_manager.clone());
#[cfg(feature = "enterprise")]
let builder = if let Some(factories) = plugins.get() {
builder.with_extra_information_table_factories(factories)
} else {
builder
};
let catalog_manager = builder.build();
let table_metadata_manager =
Self::create_table_metadata_manager(kv_backend.clone()).await?;
@@ -564,7 +571,7 @@ impl StartCommand {
// for standalone not use grpc, but get a handler to frontend grpc client without
// actually make a connection
let (frontend_client, frontend_instance_handler) =
FrontendClient::from_empty_grpc_handler();
FrontendClient::from_empty_grpc_handler(opts.query.clone());
let frontend_client = Arc::new(frontend_client);
let flow_builder = FlownodeBuilder::new(
flownode_options,

View File

@@ -30,6 +30,7 @@ use meta_srv::metasrv::MetasrvOptions;
use meta_srv::selector::SelectorType;
use metric_engine::config::EngineConfig as MetricEngineConfig;
use mito2::config::MitoConfig;
use query::options::QueryOptions;
use servers::export_metrics::ExportMetricsOption;
use servers::grpc::GrpcOptions;
use servers::http::HttpOptions;
@@ -216,12 +217,15 @@ fn test_load_flownode_example_config() {
dir: format!("{}/{}", DEFAULT_DATA_HOME, DEFAULT_LOGGING_DIR),
level: Some("info".to_string()),
otlp_endpoint: Some(DEFAULT_OTLP_HTTP_ENDPOINT.to_string()),
otlp_export_protocol: Some(common_telemetry::logging::OtlpExportProtocol::Http),
tracing_sample_ratio: Some(Default::default()),
..Default::default()
},
tracing: Default::default(),
heartbeat: Default::default(),
query: Default::default(),
// flownode deliberately use a slower query parallelism
// to avoid overwhelming the frontend with too many queries
query: QueryOptions { parallelism: 1 },
meta_client: Some(MetaClientOptions {
metasrv_addrs: vec!["127.0.0.1:3002".to_string()],
timeout: Duration::from_secs(3),

View File

@@ -78,7 +78,7 @@ pub const INFORMATION_SCHEMA_ROUTINES_TABLE_ID: u32 = 21;
pub const INFORMATION_SCHEMA_SCHEMA_PRIVILEGES_TABLE_ID: u32 = 22;
/// id for information_schema.TABLE_PRIVILEGES
pub const INFORMATION_SCHEMA_TABLE_PRIVILEGES_TABLE_ID: u32 = 23;
/// id for information_schema.TRIGGERS
/// id for information_schema.TRIGGERS (for mysql)
pub const INFORMATION_SCHEMA_TRIGGERS_TABLE_ID: u32 = 24;
/// id for information_schema.GLOBAL_STATUS
pub const INFORMATION_SCHEMA_GLOBAL_STATUS_TABLE_ID: u32 = 25;
@@ -104,6 +104,8 @@ pub const INFORMATION_SCHEMA_PROCEDURE_INFO_TABLE_ID: u32 = 34;
pub const INFORMATION_SCHEMA_REGION_STATISTICS_TABLE_ID: u32 = 35;
/// id for information_schema.process_list
pub const INFORMATION_SCHEMA_PROCESS_LIST_TABLE_ID: u32 = 36;
/// id for information_schema.trigger_list (for greptimedb trigger)
pub const INFORMATION_SCHEMA_TRIGGER_TABLE_ID: u32 = 37;
// ----- End of information_schema tables -----

View File

@@ -239,6 +239,48 @@ impl fmt::Display for StatusCode {
}
}
#[macro_export]
macro_rules! define_from_tonic_status {
($Error: ty, $Variant: ident) => {
impl From<tonic::Status> for $Error {
fn from(e: tonic::Status) -> Self {
use snafu::location;
fn metadata_value(e: &tonic::Status, key: &str) -> Option<String> {
e.metadata()
.get(key)
.and_then(|v| String::from_utf8(v.as_bytes().to_vec()).ok())
}
let code = metadata_value(&e, $crate::GREPTIME_DB_HEADER_ERROR_CODE)
.and_then(|s| {
if let Ok(code) = s.parse::<u32>() {
StatusCode::from_u32(code)
} else {
None
}
})
.unwrap_or_else(|| match e.code() {
tonic::Code::Cancelled => StatusCode::Cancelled,
tonic::Code::DeadlineExceeded => StatusCode::DeadlineExceeded,
_ => StatusCode::Internal,
});
let msg = metadata_value(&e, $crate::GREPTIME_DB_HEADER_ERROR_MSG)
.unwrap_or_else(|| e.message().to_string());
// TODO(LFC): Make the error variant defined automatically.
Self::$Variant {
code,
msg,
tonic_code: e.code(),
location: location!(),
}
}
}
};
}
#[macro_export]
macro_rules! define_into_tonic_status {
($Error: ty) => {
@@ -315,15 +357,6 @@ pub fn status_to_tonic_code(status_code: StatusCode) -> Code {
}
}
/// Converts tonic [Code] to [StatusCode].
pub fn convert_tonic_code_to_status_code(code: Code) -> StatusCode {
match code {
Code::Cancelled => StatusCode::Cancelled,
Code::DeadlineExceeded => StatusCode::DeadlineExceeded,
_ => StatusCode::Internal,
}
}
#[cfg(test)]
mod tests {
use strum::IntoEnumIterator;

View File

@@ -12,8 +12,8 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::fmt;
use std::sync::Arc;
use std::{env, fmt};
use common_query::error::Result;
use common_query::prelude::{Signature, Volatility};
@@ -47,7 +47,7 @@ impl Function for PGVersionFunction {
fn eval(&self, _func_ctx: &FunctionContext, _columns: &[VectorRef]) -> Result<VectorRef> {
let result = StringVector::from(vec![format!(
"PostgreSQL 16.3 GreptimeDB {}",
env!("CARGO_PKG_VERSION")
common_version::version()
)]);
Ok(Arc::new(result))
}

View File

@@ -12,8 +12,8 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::fmt;
use std::sync::Arc;
use std::{env, fmt};
use common_query::error::Result;
use common_query::prelude::{Signature, Volatility};
@@ -52,13 +52,13 @@ impl Function for VersionFunction {
"{}-greptimedb-{}",
std::env::var("GREPTIMEDB_MYSQL_SERVER_VERSION")
.unwrap_or_else(|_| "8.4.2".to_string()),
env!("CARGO_PKG_VERSION")
common_version::version()
)
}
Channel::Postgres => {
format!("16.3-greptimedb-{}", env!("CARGO_PKG_VERSION"))
format!("16.3-greptimedb-{}", common_version::version())
}
_ => env!("CARGO_PKG_VERSION").to_string(),
_ => common_version::version().to_string(),
};
let result = StringVector::from(vec![version]);
Ok(Arc::new(result))

View File

@@ -34,7 +34,7 @@ use table::requests::{
};
use crate::error::{
InvalidColumnDefSnafu, InvalidSetFulltextOptionRequestSnafu,
InvalidColumnDefSnafu, InvalidIndexOptionSnafu, InvalidSetFulltextOptionRequestSnafu,
InvalidSetSkippingIndexOptionRequestSnafu, InvalidSetTableOptionRequestSnafu,
InvalidUnsetTableOptionRequestSnafu, MissingAlterIndexOptionSnafu, MissingFieldSnafu,
MissingTimestampColumnSnafu, Result, UnknownLocationTypeSnafu,
@@ -126,18 +126,21 @@ pub fn alter_expr_to_request(table_id: TableId, expr: AlterTableExpr) -> Result<
api::v1::set_index::Options::Fulltext(f) => AlterKind::SetIndex {
options: SetIndexOptions::Fulltext {
column_name: f.column_name.clone(),
options: FulltextOptions {
enable: f.enable,
analyzer: as_fulltext_option_analyzer(
options: FulltextOptions::new(
f.enable,
as_fulltext_option_analyzer(
Analyzer::try_from(f.analyzer)
.context(InvalidSetFulltextOptionRequestSnafu)?,
),
case_sensitive: f.case_sensitive,
backend: as_fulltext_option_backend(
f.case_sensitive,
as_fulltext_option_backend(
PbFulltextBackend::try_from(f.backend)
.context(InvalidSetFulltextOptionRequestSnafu)?,
),
},
f.granularity as u32,
f.false_positive_rate,
)
.context(InvalidIndexOptionSnafu)?,
},
},
api::v1::set_index::Options::Inverted(i) => AlterKind::SetIndex {
@@ -148,13 +151,15 @@ pub fn alter_expr_to_request(table_id: TableId, expr: AlterTableExpr) -> Result<
api::v1::set_index::Options::Skipping(s) => AlterKind::SetIndex {
options: SetIndexOptions::Skipping {
column_name: s.column_name,
options: SkippingIndexOptions {
granularity: s.granularity as u32,
index_type: as_skipping_index_type(
options: SkippingIndexOptions::new(
s.granularity as u32,
s.false_positive_rate,
as_skipping_index_type(
PbSkippingIndexType::try_from(s.skipping_index_type)
.context(InvalidSetSkippingIndexOptionRequestSnafu)?,
),
},
)
.context(InvalidIndexOptionSnafu)?,
},
},
},

View File

@@ -153,6 +153,14 @@ pub enum Error {
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Invalid index option"))]
InvalidIndexOption {
#[snafu(implicit)]
location: Location,
#[snafu(source)]
error: datatypes::error::Error,
},
}
pub type Result<T> = std::result::Result<T, Error>;
@@ -180,7 +188,8 @@ impl ErrorExt for Error {
| Error::InvalidUnsetTableOptionRequest { .. }
| Error::InvalidSetFulltextOptionRequest { .. }
| Error::InvalidSetSkippingIndexOptionRequest { .. }
| Error::MissingAlterIndexOption { .. } => StatusCode::InvalidArguments,
| Error::MissingAlterIndexOption { .. }
| Error::InvalidIndexOption { .. } => StatusCode::InvalidArguments,
}
}

View File

@@ -50,7 +50,6 @@ pub mod drop_flow;
pub mod drop_table;
pub mod drop_view;
pub mod flow_meta;
mod physical_table_metadata;
pub mod table_meta;
#[cfg(any(test, feature = "testing"))]
pub mod test_util;

View File

@@ -12,32 +12,32 @@
// See the License for the specific language governing permissions and
// limitations under the License.
mod check;
mod metadata;
mod region_request;
mod table_cache_keys;
mod executor;
mod update_metadata;
mod validator;
use api::region::RegionResponse;
use async_trait::async_trait;
use common_catalog::format_full_table_name;
use common_procedure::error::{FromJsonSnafu, Result as ProcedureResult, ToJsonSnafu};
use common_procedure::{Context, LockKey, Procedure, Status};
use common_telemetry::{error, info, warn};
use futures_util::future;
pub use region_request::make_alter_region_request;
use common_telemetry::{debug, error, info, warn};
pub use executor::make_alter_region_request;
use serde::{Deserialize, Serialize};
use snafu::{ensure, ResultExt};
use snafu::ResultExt;
use store_api::metadata::ColumnMetadata;
use store_api::metric_engine_consts::ALTER_PHYSICAL_EXTENSION_KEY;
use strum::AsRefStr;
use table::metadata::TableId;
use crate::ddl::utils::{
add_peer_context_if_needed, map_to_procedure_error, sync_follower_regions,
use crate::cache_invalidator::Context as CacheContext;
use crate::ddl::alter_logical_tables::executor::AlterLogicalTablesExecutor;
use crate::ddl::alter_logical_tables::validator::{
retain_unskipped, AlterLogicalTableValidator, ValidatorResult,
};
use crate::ddl::utils::{extract_column_metadatas, map_to_procedure_error, sync_follower_regions};
use crate::ddl::DdlContext;
use crate::error::{DecodeJsonSnafu, MetadataCorruptionSnafu, Result};
use crate::error::Result;
use crate::instruction::CacheIdent;
use crate::key::table_info::TableInfoValue;
use crate::key::table_route::PhysicalTableRouteValue;
@@ -45,13 +45,38 @@ use crate::key::DeserializedValueWithBytes;
use crate::lock_key::{CatalogLock, SchemaLock, TableLock};
use crate::metrics;
use crate::rpc::ddl::AlterTableTask;
use crate::rpc::router::{find_leaders, RegionRoute};
use crate::rpc::router::RegionRoute;
pub struct AlterLogicalTablesProcedure {
pub context: DdlContext,
pub data: AlterTablesData,
}
/// Builds the validator from the [`AlterTablesData`].
fn build_validator_from_alter_table_data<'a>(
data: &'a AlterTablesData,
) -> AlterLogicalTableValidator<'a> {
let phsycial_table_id = data.physical_table_id;
let alters = data
.tasks
.iter()
.map(|task| &task.alter_table)
.collect::<Vec<_>>();
AlterLogicalTableValidator::new(phsycial_table_id, alters)
}
/// Builds the executor from the [`AlterTablesData`].
fn build_executor_from_alter_expr<'a>(data: &'a AlterTablesData) -> AlterLogicalTablesExecutor<'a> {
debug_assert_eq!(data.tasks.len(), data.table_info_values.len());
let alters = data
.tasks
.iter()
.zip(data.table_info_values.iter())
.map(|(task, table_info)| (table_info.table_info.ident.table_id, &task.alter_table))
.collect::<Vec<_>>();
AlterLogicalTablesExecutor::new(alters)
}
impl AlterLogicalTablesProcedure {
pub const TYPE_NAME: &'static str = "metasrv-procedure::AlterLogicalTables";
@@ -81,35 +106,44 @@ impl AlterLogicalTablesProcedure {
}
pub(crate) async fn on_prepare(&mut self) -> Result<Status> {
// Checks all the tasks
self.check_input_tasks()?;
// Fills the table info values
self.fill_table_info_values().await?;
// Checks the physical table, must after [fill_table_info_values]
self.check_physical_table().await?;
// Fills the physical table info
self.fill_physical_table_info().await?;
// Filter the finished tasks
let finished_tasks = self.check_finished_tasks()?;
let already_finished_count = finished_tasks
.iter()
.map(|x| if *x { 1 } else { 0 })
.sum::<usize>();
let apply_tasks_count = self.data.tasks.len();
if already_finished_count == apply_tasks_count {
let validator = build_validator_from_alter_table_data(&self.data);
let ValidatorResult {
num_skipped,
skip_alter,
table_info_values,
physical_table_info,
physical_table_route,
} = validator
.validate(&self.context.table_metadata_manager)
.await?;
let num_tasks = self.data.tasks.len();
if num_skipped == num_tasks {
info!("All the alter tasks are finished, will skip the procedure.");
let cache_ident_keys = AlterLogicalTablesExecutor::build_cache_ident_keys(
&physical_table_info,
&table_info_values
.iter()
.map(|v| v.get_inner_ref())
.collect::<Vec<_>>(),
);
self.data.table_cache_keys_to_invalidate = cache_ident_keys;
// Re-invalidate the table cache
self.data.state = AlterTablesState::InvalidateTableCache;
return Ok(Status::executing(true));
} else if already_finished_count > 0 {
} else if num_skipped > 0 {
info!(
"There are {} alter tasks, {} of them were already finished.",
apply_tasks_count, already_finished_count
num_tasks, num_skipped
);
}
self.filter_task(&finished_tasks)?;
// Next state
// Updates the procedure state.
retain_unskipped(&mut self.data.tasks, &skip_alter);
self.data.physical_table_info = Some(physical_table_info);
self.data.physical_table_route = Some(physical_table_route);
self.data.table_info_values = table_info_values;
debug_assert_eq!(self.data.tasks.len(), self.data.table_info_values.len());
self.data.state = AlterTablesState::SubmitAlterRegionRequests;
Ok(Status::executing(true))
}
@@ -117,57 +151,21 @@ impl AlterLogicalTablesProcedure {
pub(crate) async fn on_submit_alter_region_requests(&mut self) -> Result<Status> {
// Safety: we have checked the state in on_prepare
let physical_table_route = &self.data.physical_table_route.as_ref().unwrap();
let leaders = find_leaders(&physical_table_route.region_routes);
let mut alter_region_tasks = Vec::with_capacity(leaders.len());
let executor = build_executor_from_alter_expr(&self.data);
let mut results = executor
.on_alter_regions(
&self.context.node_manager,
&physical_table_route.region_routes,
)
.await?;
for peer in leaders {
let requester = self.context.node_manager.datanode(&peer).await;
let request = self.make_request(&peer, &physical_table_route.region_routes)?;
alter_region_tasks.push(async move {
requester
.handle(request)
.await
.map_err(add_peer_context_if_needed(peer))
});
}
let mut results = future::join_all(alter_region_tasks)
.await
.into_iter()
.collect::<Result<Vec<_>>>()?;
// Collects responses from datanodes.
let phy_raw_schemas = results
.iter_mut()
.map(|res| res.extensions.remove(ALTER_PHYSICAL_EXTENSION_KEY))
.collect::<Vec<_>>();
if phy_raw_schemas.is_empty() {
self.submit_sync_region_requests(results, &physical_table_route.region_routes)
.await;
self.data.state = AlterTablesState::UpdateMetadata;
return Ok(Status::executing(true));
}
// Verify all the physical schemas are the same
// Safety: previous check ensures this vec is not empty
let first = phy_raw_schemas.first().unwrap();
ensure!(
phy_raw_schemas.iter().all(|x| x == first),
MetadataCorruptionSnafu {
err_msg: "The physical schemas from datanodes are not the same."
}
);
// Decodes the physical raw schemas
if let Some(phy_raw_schema) = first {
self.data.physical_columns =
ColumnMetadata::decode_list(phy_raw_schema).context(DecodeJsonSnafu)?;
if let Some(column_metadatas) =
extract_column_metadatas(&mut results, ALTER_PHYSICAL_EXTENSION_KEY)?
{
self.data.physical_columns = column_metadatas;
} else {
warn!("altering logical table result doesn't contains extension key `{ALTER_PHYSICAL_EXTENSION_KEY}`,leaving the physical table's schema unchanged");
}
self.submit_sync_region_requests(results, &physical_table_route.region_routes)
.await;
self.data.state = AlterTablesState::UpdateMetadata;
@@ -183,7 +181,7 @@ impl AlterLogicalTablesProcedure {
if let Err(err) = sync_follower_regions(
&self.context,
self.data.physical_table_id,
results,
&results,
region_routes,
table_info.meta.engine.as_str(),
)
@@ -200,7 +198,18 @@ impl AlterLogicalTablesProcedure {
self.update_physical_table_metadata().await?;
self.update_logical_tables_metadata().await?;
self.data.build_cache_keys_to_invalidate();
let logical_table_info_values = self
.data
.table_info_values
.iter()
.map(|v| v.get_inner_ref())
.collect::<Vec<_>>();
let cache_ident_keys = AlterLogicalTablesExecutor::build_cache_ident_keys(
self.data.physical_table_info.as_ref().unwrap(),
&logical_table_info_values,
);
self.data.table_cache_keys_to_invalidate = cache_ident_keys;
self.data.clear_metadata_fields();
self.data.state = AlterTablesState::InvalidateTableCache;
@@ -210,9 +219,16 @@ impl AlterLogicalTablesProcedure {
pub(crate) async fn on_invalidate_table_cache(&mut self) -> Result<Status> {
let to_invalidate = &self.data.table_cache_keys_to_invalidate;
let ctx = CacheContext {
subject: Some(format!(
"Invalidate table cache by altering logical tables, physical_table_id: {}",
self.data.physical_table_id,
)),
};
self.context
.cache_invalidator
.invalidate(&Default::default(), to_invalidate)
.invalidate(&ctx, to_invalidate)
.await?;
Ok(Status::done())
}
@@ -232,6 +248,10 @@ impl Procedure for AlterLogicalTablesProcedure {
let _timer = metrics::METRIC_META_PROCEDURE_ALTER_TABLE
.with_label_values(&[step])
.start_timer();
debug!(
"Executing alter logical tables procedure, state: {:?}",
state
);
match state {
AlterTablesState::Prepare => self.on_prepare().await,

View File

@@ -1,136 +0,0 @@
// 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::HashSet;
use api::v1::alter_table_expr::Kind;
use snafu::{ensure, OptionExt};
use crate::ddl::alter_logical_tables::AlterLogicalTablesProcedure;
use crate::error::{AlterLogicalTablesInvalidArgumentsSnafu, Result};
use crate::key::table_info::TableInfoValue;
use crate::key::table_route::TableRouteValue;
use crate::rpc::ddl::AlterTableTask;
impl AlterLogicalTablesProcedure {
pub(crate) fn check_input_tasks(&self) -> Result<()> {
self.check_schema()?;
self.check_alter_kind()?;
Ok(())
}
pub(crate) async fn check_physical_table(&self) -> Result<()> {
let table_route_manager = self.context.table_metadata_manager.table_route_manager();
let table_ids = self
.data
.table_info_values
.iter()
.map(|v| v.table_info.ident.table_id)
.collect::<Vec<_>>();
let table_routes = table_route_manager
.table_route_storage()
.batch_get(&table_ids)
.await?;
let physical_table_id = self.data.physical_table_id;
let is_same_physical_table = table_routes.iter().all(|r| {
if let Some(TableRouteValue::Logical(r)) = r {
r.physical_table_id() == physical_table_id
} else {
false
}
});
ensure!(
is_same_physical_table,
AlterLogicalTablesInvalidArgumentsSnafu {
err_msg: "All the tasks should have the same physical table id"
}
);
Ok(())
}
pub(crate) fn check_finished_tasks(&self) -> Result<Vec<bool>> {
let task = &self.data.tasks;
let table_info_values = &self.data.table_info_values;
Ok(task
.iter()
.zip(table_info_values.iter())
.map(|(task, table)| Self::check_finished_task(task, table))
.collect())
}
// Checks if the schemas of the tasks are the same
fn check_schema(&self) -> Result<()> {
let is_same_schema = self.data.tasks.windows(2).all(|pair| {
pair[0].alter_table.catalog_name == pair[1].alter_table.catalog_name
&& pair[0].alter_table.schema_name == pair[1].alter_table.schema_name
});
ensure!(
is_same_schema,
AlterLogicalTablesInvalidArgumentsSnafu {
err_msg: "Schemas of the tasks are not the same"
}
);
Ok(())
}
fn check_alter_kind(&self) -> Result<()> {
for task in &self.data.tasks {
let kind = task.alter_table.kind.as_ref().context(
AlterLogicalTablesInvalidArgumentsSnafu {
err_msg: "Alter kind is missing",
},
)?;
let Kind::AddColumns(_) = kind else {
return AlterLogicalTablesInvalidArgumentsSnafu {
err_msg: "Only support add columns operation",
}
.fail();
};
}
Ok(())
}
fn check_finished_task(task: &AlterTableTask, table: &TableInfoValue) -> bool {
let columns = table
.table_info
.meta
.schema
.column_schemas
.iter()
.map(|c| &c.name)
.collect::<HashSet<_>>();
let Some(kind) = task.alter_table.kind.as_ref() else {
return true; // Never get here since we have checked it in `check_alter_kind`
};
let Kind::AddColumns(add_columns) = kind else {
return true; // Never get here since we have checked it in `check_alter_kind`
};
// We only check that all columns have been finished. That is to say,
// if one part is finished but another part is not, it will be considered
// unfinished.
add_columns
.add_columns
.iter()
.map(|add_column| add_column.column_def.as_ref().map(|c| &c.name))
.all(|column| column.map(|c| columns.contains(c)).unwrap_or(false))
}
}

View File

@@ -0,0 +1,216 @@
// 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 api::region::RegionResponse;
use api::v1::alter_table_expr::Kind;
use api::v1::region::{
alter_request, region_request, AddColumn, AddColumns, AlterRequest, AlterRequests,
RegionColumnDef, RegionRequest, RegionRequestHeader,
};
use api::v1::{self, AlterTableExpr};
use common_telemetry::tracing_context::TracingContext;
use common_telemetry::{debug, warn};
use futures::future;
use store_api::metadata::ColumnMetadata;
use store_api::storage::{RegionId, RegionNumber, TableId};
use crate::ddl::utils::{add_peer_context_if_needed, raw_table_info};
use crate::error::Result;
use crate::instruction::CacheIdent;
use crate::key::table_info::TableInfoValue;
use crate::key::{DeserializedValueWithBytes, RegionDistribution, TableMetadataManagerRef};
use crate::node_manager::NodeManagerRef;
use crate::rpc::router::{find_leaders, region_distribution, RegionRoute};
/// [AlterLogicalTablesExecutor] performs:
/// - Alters logical regions on the datanodes.
/// - Updates table metadata for alter table operation.
pub struct AlterLogicalTablesExecutor<'a> {
/// The alter table expressions.
///
/// The first element is the logical table id, the second element is the alter table expression.
alters: Vec<(TableId, &'a AlterTableExpr)>,
}
impl<'a> AlterLogicalTablesExecutor<'a> {
pub fn new(alters: Vec<(TableId, &'a AlterTableExpr)>) -> Self {
Self { alters }
}
/// Alters logical regions on the datanodes.
pub(crate) async fn on_alter_regions(
&self,
node_manager: &NodeManagerRef,
region_routes: &[RegionRoute],
) -> Result<Vec<RegionResponse>> {
let region_distribution = region_distribution(region_routes);
let leaders = find_leaders(region_routes)
.into_iter()
.map(|p| (p.id, p))
.collect::<HashMap<_, _>>();
let mut alter_region_tasks = Vec::with_capacity(leaders.len());
for (datanode_id, region_role_set) in region_distribution {
if region_role_set.leader_regions.is_empty() {
continue;
}
// Safety: must exists.
let peer = leaders.get(&datanode_id).unwrap();
let requester = node_manager.datanode(peer).await;
let requests = self.make_alter_region_request(&region_role_set.leader_regions);
let requester = requester.clone();
let peer = peer.clone();
debug!("Sending alter region requests to datanode {}", peer);
alter_region_tasks.push(async move {
requester
.handle(make_request(requests))
.await
.map_err(add_peer_context_if_needed(peer))
});
}
future::join_all(alter_region_tasks)
.await
.into_iter()
.collect::<Result<Vec<_>>>()
}
fn make_alter_region_request(&self, region_numbers: &[RegionNumber]) -> AlterRequests {
let mut requests = Vec::with_capacity(region_numbers.len() * self.alters.len());
for (table_id, alter) in self.alters.iter() {
for region_number in region_numbers {
let region_id = RegionId::new(*table_id, *region_number);
let request = make_alter_region_request(region_id, alter);
requests.push(request);
}
}
AlterRequests { requests }
}
/// Updates table metadata for alter table operation.
///
/// ## Panic:
/// - If the region distribution is not set when updating table metadata.
pub(crate) async fn on_alter_metadata(
physical_table_id: TableId,
table_metadata_manager: &TableMetadataManagerRef,
current_table_info_value: &DeserializedValueWithBytes<TableInfoValue>,
region_distribution: RegionDistribution,
physical_columns: &[ColumnMetadata],
) -> Result<()> {
if physical_columns.is_empty() {
warn!("No physical columns found, leaving the physical table's schema unchanged when altering logical tables");
return Ok(());
}
let table_ref = current_table_info_value.table_ref();
let table_id = physical_table_id;
// Generates new table info
let old_raw_table_info = current_table_info_value.table_info.clone();
let new_raw_table_info =
raw_table_info::build_new_physical_table_info(old_raw_table_info, physical_columns);
debug!(
"Starting update table: {} metadata, table_id: {}, new table info: {:?}",
table_ref, table_id, new_raw_table_info
);
table_metadata_manager
.update_table_info(
current_table_info_value,
Some(region_distribution),
new_raw_table_info,
)
.await?;
Ok(())
}
/// Builds the cache ident keys for the alter logical tables.
///
/// The cache ident keys are:
/// - The table id of the logical tables.
/// - The table name of the logical tables.
/// - The table id of the physical table.
pub(crate) fn build_cache_ident_keys(
physical_table_info: &TableInfoValue,
logical_table_info_values: &[&TableInfoValue],
) -> Vec<CacheIdent> {
let mut cache_keys = Vec::with_capacity(logical_table_info_values.len() * 2 + 2);
cache_keys.extend(logical_table_info_values.iter().flat_map(|table| {
vec![
CacheIdent::TableId(table.table_info.ident.table_id),
CacheIdent::TableName(table.table_name()),
]
}));
cache_keys.push(CacheIdent::TableId(
physical_table_info.table_info.ident.table_id,
));
cache_keys.push(CacheIdent::TableName(physical_table_info.table_name()));
cache_keys
}
}
fn make_request(alter_requests: AlterRequests) -> RegionRequest {
RegionRequest {
header: Some(RegionRequestHeader {
tracing_context: TracingContext::from_current_span().to_w3c(),
..Default::default()
}),
body: Some(region_request::Body::Alters(alter_requests)),
}
}
/// Makes an alter region request.
pub fn make_alter_region_request(
region_id: RegionId,
alter_table_expr: &AlterTableExpr,
) -> AlterRequest {
let region_id = region_id.as_u64();
let kind = match &alter_table_expr.kind {
Some(Kind::AddColumns(add_columns)) => Some(alter_request::Kind::AddColumns(
to_region_add_columns(add_columns),
)),
_ => unreachable!(), // Safety: we have checked the kind in check_input_tasks
};
AlterRequest {
region_id,
schema_version: 0,
kind,
}
}
fn to_region_add_columns(add_columns: &v1::AddColumns) -> AddColumns {
let add_columns = add_columns
.add_columns
.iter()
.map(|add_column| {
let region_column_def = RegionColumnDef {
column_def: add_column.column_def.clone(),
..Default::default() // other fields are not used in alter logical table
};
AddColumn {
column_def: Some(region_column_def),
..Default::default() // other fields are not used in alter logical table
}
})
.collect();
AddColumns { add_columns }
}

View File

@@ -1,158 +0,0 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use common_catalog::format_full_table_name;
use snafu::OptionExt;
use table::metadata::TableId;
use crate::ddl::alter_logical_tables::AlterLogicalTablesProcedure;
use crate::error::{
AlterLogicalTablesInvalidArgumentsSnafu, Result, TableInfoNotFoundSnafu, TableNotFoundSnafu,
TableRouteNotFoundSnafu,
};
use crate::key::table_info::TableInfoValue;
use crate::key::table_name::TableNameKey;
use crate::key::table_route::TableRouteValue;
use crate::key::DeserializedValueWithBytes;
use crate::rpc::ddl::AlterTableTask;
impl AlterLogicalTablesProcedure {
pub(crate) fn filter_task(&mut self, finished_tasks: &[bool]) -> Result<()> {
debug_assert_eq!(finished_tasks.len(), self.data.tasks.len());
debug_assert_eq!(finished_tasks.len(), self.data.table_info_values.len());
self.data.tasks = self
.data
.tasks
.drain(..)
.zip(finished_tasks.iter())
.filter_map(|(task, finished)| if *finished { None } else { Some(task) })
.collect();
self.data.table_info_values = self
.data
.table_info_values
.drain(..)
.zip(finished_tasks.iter())
.filter_map(|(table_info_value, finished)| {
if *finished {
None
} else {
Some(table_info_value)
}
})
.collect();
Ok(())
}
pub(crate) async fn fill_physical_table_info(&mut self) -> Result<()> {
let (physical_table_info, physical_table_route) = self
.context
.table_metadata_manager
.get_full_table_info(self.data.physical_table_id)
.await?;
let physical_table_info = physical_table_info.with_context(|| TableInfoNotFoundSnafu {
table: format!("table id - {}", self.data.physical_table_id),
})?;
let physical_table_route = physical_table_route
.context(TableRouteNotFoundSnafu {
table_id: self.data.physical_table_id,
})?
.into_inner();
self.data.physical_table_info = Some(physical_table_info);
let TableRouteValue::Physical(physical_table_route) = physical_table_route else {
return AlterLogicalTablesInvalidArgumentsSnafu {
err_msg: format!(
"expected a physical table but got a logical table: {:?}",
self.data.physical_table_id
),
}
.fail();
};
self.data.physical_table_route = Some(physical_table_route);
Ok(())
}
pub(crate) async fn fill_table_info_values(&mut self) -> Result<()> {
let table_ids = self.get_all_table_ids().await?;
let table_info_values = self.get_all_table_info_values(&table_ids).await?;
debug_assert_eq!(table_info_values.len(), self.data.tasks.len());
self.data.table_info_values = table_info_values;
Ok(())
}
async fn get_all_table_info_values(
&self,
table_ids: &[TableId],
) -> Result<Vec<DeserializedValueWithBytes<TableInfoValue>>> {
let table_info_manager = self.context.table_metadata_manager.table_info_manager();
let mut table_info_map = table_info_manager.batch_get_raw(table_ids).await?;
let mut table_info_values = Vec::with_capacity(table_ids.len());
for (table_id, task) in table_ids.iter().zip(self.data.tasks.iter()) {
let table_info_value =
table_info_map
.remove(table_id)
.with_context(|| TableInfoNotFoundSnafu {
table: extract_table_name(task),
})?;
table_info_values.push(table_info_value);
}
Ok(table_info_values)
}
async fn get_all_table_ids(&self) -> Result<Vec<TableId>> {
let table_name_manager = self.context.table_metadata_manager.table_name_manager();
let table_name_keys = self
.data
.tasks
.iter()
.map(|task| extract_table_name_key(task))
.collect();
let table_name_values = table_name_manager.batch_get(table_name_keys).await?;
let mut table_ids = Vec::with_capacity(table_name_values.len());
for (value, task) in table_name_values.into_iter().zip(self.data.tasks.iter()) {
let table_id = value
.with_context(|| TableNotFoundSnafu {
table_name: extract_table_name(task),
})?
.table_id();
table_ids.push(table_id);
}
Ok(table_ids)
}
}
#[inline]
fn extract_table_name(task: &AlterTableTask) -> String {
format_full_table_name(
&task.alter_table.catalog_name,
&task.alter_table.schema_name,
&task.alter_table.table_name,
)
}
#[inline]
fn extract_table_name_key(task: &AlterTableTask) -> TableNameKey {
TableNameKey::new(
&task.alter_table.catalog_name,
&task.alter_table.schema_name,
&task.alter_table.table_name,
)
}

View File

@@ -1,113 +0,0 @@
// 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::alter_table_expr::Kind;
use api::v1::region::{
alter_request, region_request, AddColumn, AddColumns, AlterRequest, AlterRequests,
RegionColumnDef, RegionRequest, RegionRequestHeader,
};
use api::v1::{self, AlterTableExpr};
use common_telemetry::tracing_context::TracingContext;
use store_api::storage::RegionId;
use crate::ddl::alter_logical_tables::AlterLogicalTablesProcedure;
use crate::error::Result;
use crate::peer::Peer;
use crate::rpc::router::{find_leader_regions, RegionRoute};
impl AlterLogicalTablesProcedure {
pub(crate) fn make_request(
&self,
peer: &Peer,
region_routes: &[RegionRoute],
) -> Result<RegionRequest> {
let alter_requests = self.make_alter_region_requests(peer, region_routes)?;
let request = RegionRequest {
header: Some(RegionRequestHeader {
tracing_context: TracingContext::from_current_span().to_w3c(),
..Default::default()
}),
body: Some(region_request::Body::Alters(alter_requests)),
};
Ok(request)
}
fn make_alter_region_requests(
&self,
peer: &Peer,
region_routes: &[RegionRoute],
) -> Result<AlterRequests> {
let tasks = &self.data.tasks;
let regions_on_this_peer = find_leader_regions(region_routes, peer);
let mut requests = Vec::with_capacity(tasks.len() * regions_on_this_peer.len());
for (task, table) in self
.data
.tasks
.iter()
.zip(self.data.table_info_values.iter())
{
for region_number in &regions_on_this_peer {
let region_id = RegionId::new(table.table_info.ident.table_id, *region_number);
let request = make_alter_region_request(
region_id,
&task.alter_table,
table.table_info.ident.version,
);
requests.push(request);
}
}
Ok(AlterRequests { requests })
}
}
/// Makes an alter region request.
pub fn make_alter_region_request(
region_id: RegionId,
alter_table_expr: &AlterTableExpr,
schema_version: u64,
) -> AlterRequest {
let region_id = region_id.as_u64();
let kind = match &alter_table_expr.kind {
Some(Kind::AddColumns(add_columns)) => Some(alter_request::Kind::AddColumns(
to_region_add_columns(add_columns),
)),
_ => unreachable!(), // Safety: we have checked the kind in check_input_tasks
};
AlterRequest {
region_id,
schema_version,
kind,
}
}
fn to_region_add_columns(add_columns: &v1::AddColumns) -> AddColumns {
let add_columns = add_columns
.add_columns
.iter()
.map(|add_column| {
let region_column_def = RegionColumnDef {
column_def: add_column.column_def.clone(),
..Default::default() // other fields are not used in alter logical table
};
AddColumn {
column_def: Some(region_column_def),
..Default::default() // other fields are not used in alter logical table
}
})
.collect();
AddColumns { add_columns }
}

View File

@@ -1,50 +0,0 @@
// 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 table::metadata::RawTableInfo;
use table::table_name::TableName;
use crate::ddl::alter_logical_tables::AlterTablesData;
use crate::instruction::CacheIdent;
impl AlterTablesData {
pub(crate) fn build_cache_keys_to_invalidate(&mut self) {
let mut cache_keys = self
.table_info_values
.iter()
.flat_map(|table| {
vec![
CacheIdent::TableId(table.table_info.ident.table_id),
CacheIdent::TableName(extract_table_name(&table.table_info)),
]
})
.collect::<Vec<_>>();
cache_keys.push(CacheIdent::TableId(self.physical_table_id));
// Safety: physical_table_info already filled in previous steps
let physical_table_info = &self.physical_table_info.as_ref().unwrap().table_info;
cache_keys.push(CacheIdent::TableName(extract_table_name(
physical_table_info,
)));
self.table_cache_keys_to_invalidate = cache_keys;
}
}
fn extract_table_name(table_info: &RawTableInfo) -> TableName {
TableName::new(
&table_info.catalog_name,
&table_info.schema_name,
&table_info.name,
)
}

View File

@@ -13,41 +13,35 @@
// limitations under the License.
use common_grpc_expr::alter_expr_to_request;
use common_telemetry::warn;
use itertools::Itertools;
use snafu::ResultExt;
use table::metadata::{RawTableInfo, TableInfo};
use crate::ddl::alter_logical_tables::executor::AlterLogicalTablesExecutor;
use crate::ddl::alter_logical_tables::AlterLogicalTablesProcedure;
use crate::ddl::physical_table_metadata;
use crate::error;
use crate::error::{ConvertAlterTableRequestSnafu, Result};
use crate::key::table_info::TableInfoValue;
use crate::key::DeserializedValueWithBytes;
use crate::rpc::ddl::AlterTableTask;
use crate::rpc::router::region_distribution;
impl AlterLogicalTablesProcedure {
pub(crate) async fn update_physical_table_metadata(&mut self) -> Result<()> {
if self.data.physical_columns.is_empty() {
warn!("No physical columns found, leaving the physical table's schema unchanged when altering logical tables");
return Ok(());
}
// Safety: must exist.
let physical_table_info = self.data.physical_table_info.as_ref().unwrap();
let physical_table_route = self.data.physical_table_route.as_ref().unwrap();
let region_distribution = region_distribution(&physical_table_route.region_routes);
// Generates new table info
let old_raw_table_info = physical_table_info.table_info.clone();
let new_raw_table_info = physical_table_metadata::build_new_physical_table_info(
old_raw_table_info,
// Updates physical table's metadata.
AlterLogicalTablesExecutor::on_alter_metadata(
self.data.physical_table_id,
&self.context.table_metadata_manager,
physical_table_info,
region_distribution,
&self.data.physical_columns,
);
// Updates physical table's metadata, and we don't need to touch per-region settings.
self.context
.table_metadata_manager
.update_table_info(physical_table_info, None, new_raw_table_info)
.await?;
)
.await?;
Ok(())
}

View File

@@ -0,0 +1,284 @@
// 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::HashSet;
use api::v1::alter_table_expr::Kind;
use api::v1::AlterTableExpr;
use snafu::{ensure, OptionExt};
use store_api::storage::TableId;
use table::table_reference::TableReference;
use crate::ddl::utils::table_id::get_all_table_ids_by_names;
use crate::ddl::utils::table_info::get_all_table_info_values_by_table_ids;
use crate::error::{
AlterLogicalTablesInvalidArgumentsSnafu, Result, TableInfoNotFoundSnafu,
TableRouteNotFoundSnafu,
};
use crate::key::table_info::TableInfoValue;
use crate::key::table_route::{PhysicalTableRouteValue, TableRouteManager, TableRouteValue};
use crate::key::{DeserializedValueWithBytes, TableMetadataManagerRef};
/// [AlterLogicalTableValidator] validates the alter logical expressions.
pub struct AlterLogicalTableValidator<'a> {
physical_table_id: TableId,
alters: Vec<&'a AlterTableExpr>,
}
impl<'a> AlterLogicalTableValidator<'a> {
pub fn new(physical_table_id: TableId, alters: Vec<&'a AlterTableExpr>) -> Self {
Self {
physical_table_id,
alters,
}
}
/// Validates all alter table expressions have the same schema and catalog.
fn validate_schema(&self) -> Result<()> {
let is_same_schema = self.alters.windows(2).all(|pair| {
pair[0].catalog_name == pair[1].catalog_name
&& pair[0].schema_name == pair[1].schema_name
});
ensure!(
is_same_schema,
AlterLogicalTablesInvalidArgumentsSnafu {
err_msg: "Schemas of the alter table expressions are not the same"
}
);
Ok(())
}
/// Validates that all alter table expressions are of the supported kind.
/// Currently only supports `AddColumns` operations.
fn validate_alter_kind(&self) -> Result<()> {
for alter in &self.alters {
let kind = alter
.kind
.as_ref()
.context(AlterLogicalTablesInvalidArgumentsSnafu {
err_msg: "Alter kind is missing",
})?;
let Kind::AddColumns(_) = kind else {
return AlterLogicalTablesInvalidArgumentsSnafu {
err_msg: "Only support add columns operation",
}
.fail();
};
}
Ok(())
}
fn table_names(&self) -> Vec<TableReference> {
self.alters
.iter()
.map(|alter| {
TableReference::full(&alter.catalog_name, &alter.schema_name, &alter.table_name)
})
.collect()
}
/// Validates that the physical table info and route exist.
///
/// This method performs the following validations:
/// 1. Retrieves the full table info and route for the given physical table id
/// 2. Ensures the table info and table route exists
/// 3. Verifies that the table route is actually a physical table route, not a logical one
///
/// Returns a tuple containing the validated table info and physical table route.
async fn validate_physical_table(
&self,
table_metadata_manager: &TableMetadataManagerRef,
) -> Result<(
DeserializedValueWithBytes<TableInfoValue>,
PhysicalTableRouteValue,
)> {
let (table_info, table_route) = table_metadata_manager
.get_full_table_info(self.physical_table_id)
.await?;
let table_info = table_info.with_context(|| TableInfoNotFoundSnafu {
table: format!("table id - {}", self.physical_table_id),
})?;
let physical_table_route = table_route
.context(TableRouteNotFoundSnafu {
table_id: self.physical_table_id,
})?
.into_inner();
let TableRouteValue::Physical(table_route) = physical_table_route else {
return AlterLogicalTablesInvalidArgumentsSnafu {
err_msg: format!(
"expected a physical table but got a logical table: {:?}",
self.physical_table_id
),
}
.fail();
};
Ok((table_info, table_route))
}
/// Validates that all logical table routes have the same physical table id.
///
/// This method performs the following validations:
/// 1. Retrieves table routes for all the given table ids.
/// 2. Ensures that all retrieved routes are logical table routes (not physical)
/// 3. Verifies that all logical table routes reference the same physical table id.
/// 4. Returns an error if any route is not logical or references a different physical table.
async fn validate_logical_table_routes(
&self,
table_route_manager: &TableRouteManager,
table_ids: &[TableId],
) -> Result<()> {
let table_routes = table_route_manager
.table_route_storage()
.batch_get(table_ids)
.await?;
let physical_table_id = self.physical_table_id;
let is_same_physical_table = table_routes.iter().all(|r| {
if let Some(TableRouteValue::Logical(r)) = r {
r.physical_table_id() == physical_table_id
} else {
false
}
});
ensure!(
is_same_physical_table,
AlterLogicalTablesInvalidArgumentsSnafu {
err_msg: "All the tasks should have the same physical table id"
}
);
Ok(())
}
/// Validates the alter logical expressions.
///
/// This method performs the following validations:
/// 1. Validates that all alter table expressions have the same schema and catalog.
/// 2. Validates that all alter table expressions are of the supported kind.
/// 3. Validates that the physical table info and route exist.
/// 4. Validates that all logical table routes have the same physical table id.
///
/// Returns a [ValidatorResult] containing the validation results.
pub async fn validate(
&self,
table_metadata_manager: &TableMetadataManagerRef,
) -> Result<ValidatorResult> {
self.validate_schema()?;
self.validate_alter_kind()?;
let (physical_table_info, physical_table_route) =
self.validate_physical_table(table_metadata_manager).await?;
let table_names = self.table_names();
let table_ids =
get_all_table_ids_by_names(table_metadata_manager.table_name_manager(), &table_names)
.await?;
let mut table_info_values = get_all_table_info_values_by_table_ids(
table_metadata_manager.table_info_manager(),
&table_ids,
&table_names,
)
.await?;
self.validate_logical_table_routes(
table_metadata_manager.table_route_manager(),
&table_ids,
)
.await?;
let skip_alter = self
.alters
.iter()
.zip(table_info_values.iter())
.map(|(task, table)| skip_alter_logical_region(task, table))
.collect::<Vec<_>>();
retain_unskipped(&mut table_info_values, &skip_alter);
let num_skipped = skip_alter.iter().filter(|&&x| x).count();
Ok(ValidatorResult {
num_skipped,
skip_alter,
table_info_values,
physical_table_info,
physical_table_route,
})
}
}
/// The result of the validator.
pub(crate) struct ValidatorResult {
pub(crate) num_skipped: usize,
pub(crate) skip_alter: Vec<bool>,
pub(crate) table_info_values: Vec<DeserializedValueWithBytes<TableInfoValue>>,
pub(crate) physical_table_info: DeserializedValueWithBytes<TableInfoValue>,
pub(crate) physical_table_route: PhysicalTableRouteValue,
}
/// Retains the elements that are not skipped.
pub(crate) fn retain_unskipped<T>(target: &mut Vec<T>, skipped: &[bool]) {
debug_assert_eq!(target.len(), skipped.len());
let mut iter = skipped.iter();
target.retain(|_| !iter.next().unwrap());
}
/// Returns true if does not required to alter the logical region.
fn skip_alter_logical_region(alter: &AlterTableExpr, table: &TableInfoValue) -> bool {
let existing_columns = table
.table_info
.meta
.schema
.column_schemas
.iter()
.map(|c| &c.name)
.collect::<HashSet<_>>();
let Some(kind) = alter.kind.as_ref() else {
return true; // Never get here since we have checked it in `validate_alter_kind`
};
let Kind::AddColumns(add_columns) = kind else {
return true; // Never get here since we have checked it in `validate_alter_kind`
};
// We only check that all columns have been finished. That is to say,
// if one part is finished but another part is not, it will be considered
// unfinished.
add_columns
.add_columns
.iter()
.map(|add_column| add_column.column_def.as_ref().map(|c| &c.name))
.all(|column| {
column
.map(|c| existing_columns.contains(c))
.unwrap_or(false)
})
}
#[cfg(test)]
mod tests {
use super::*;
#[test]
fn test_retain_unskipped() {
let mut target = vec![1, 2, 3, 4, 5];
let skipped = vec![false, true, false, true, false];
retain_unskipped(&mut target, &skipped);
assert_eq!(target, vec![1, 3, 5]);
}
}

View File

@@ -12,10 +12,9 @@
// See the License for the specific language governing permissions and
// limitations under the License.
mod check;
mod executor;
mod metadata;
mod region_request;
mod update_metadata;
use std::vec;
@@ -29,30 +28,29 @@ use common_procedure::{
Context as ProcedureContext, ContextProvider, Error as ProcedureError, LockKey, PoisonKey,
PoisonKeys, Procedure, ProcedureId, Status, StringKey,
};
use common_telemetry::{debug, error, info};
use futures::future::{self};
use common_telemetry::{error, info, warn};
use serde::{Deserialize, Serialize};
use snafu::{ensure, ResultExt};
use store_api::storage::RegionId;
use store_api::metadata::ColumnMetadata;
use store_api::metric_engine_consts::TABLE_COLUMN_METADATA_EXTENSION_KEY;
use strum::AsRefStr;
use table::metadata::{RawTableInfo, TableId, TableInfo};
use table::table_reference::TableReference;
use crate::cache_invalidator::Context;
use crate::ddl::alter_table::executor::AlterTableExecutor;
use crate::ddl::utils::{
add_peer_context_if_needed, handle_multiple_results, map_to_procedure_error,
extract_column_metadatas, handle_multiple_results, map_to_procedure_error,
sync_follower_regions, MultipleResults,
};
use crate::ddl::DdlContext;
use crate::error::{AbortProcedureSnafu, NoLeaderSnafu, PutPoisonSnafu, Result, RetryLaterSnafu};
use crate::instruction::CacheIdent;
use crate::key::table_info::TableInfoValue;
use crate::key::{DeserializedValueWithBytes, RegionDistribution};
use crate::lock_key::{CatalogLock, SchemaLock, TableLock, TableNameLock};
use crate::metrics;
use crate::poison_key::table_poison_key;
use crate::rpc::ddl::AlterTableTask;
use crate::rpc::router::{find_leader_regions, find_leaders, region_distribution, RegionRoute};
use crate::rpc::router::{find_leaders, region_distribution, RegionRoute};
/// The alter table procedure
pub struct AlterTableProcedure {
@@ -64,6 +62,24 @@ pub struct AlterTableProcedure {
/// If we recover the procedure from json, then the table info value is not cached.
/// But we already validated it in the prepare step.
new_table_info: Option<TableInfo>,
/// The alter table executor.
executor: AlterTableExecutor,
}
/// Builds the executor from the [`AlterTableData`].
///
/// # Panics
/// - If the alter kind is not set.
fn build_executor_from_alter_expr(alter_data: &AlterTableData) -> AlterTableExecutor {
let table_name = alter_data.table_ref().into();
let table_id = alter_data.table_id;
let alter_kind = alter_data.task.alter_table.kind.as_ref().unwrap();
let new_table_name = if let Kind::RenameTable(RenameTable { new_table_name }) = alter_kind {
Some(new_table_name.to_string())
} else {
None
};
AlterTableExecutor::new(table_name, table_id, new_table_name)
}
impl AlterTableProcedure {
@@ -71,33 +87,42 @@ impl AlterTableProcedure {
pub fn new(table_id: TableId, task: AlterTableTask, context: DdlContext) -> Result<Self> {
task.validate()?;
let data = AlterTableData::new(task, table_id);
let executor = build_executor_from_alter_expr(&data);
Ok(Self {
context,
data: AlterTableData::new(task, table_id),
data,
new_table_info: None,
executor,
})
}
pub fn from_json(json: &str, context: DdlContext) -> ProcedureResult<Self> {
let data: AlterTableData = serde_json::from_str(json).context(FromJsonSnafu)?;
let executor = build_executor_from_alter_expr(&data);
Ok(AlterTableProcedure {
context,
data,
new_table_info: None,
executor,
})
}
// Checks whether the table exists.
pub(crate) async fn on_prepare(&mut self) -> Result<Status> {
self.check_alter().await?;
self.executor
.on_prepare(&self.context.table_metadata_manager)
.await?;
self.fill_table_info().await?;
// Validates the request and builds the new table info.
// We need to build the new table info here because we should ensure the alteration
// is valid in `UpdateMeta` state as we already altered the region.
// Safety: `fill_table_info()` already set it.
// Safety: filled in `fill_table_info`.
let table_info_value = self.data.table_info_value.as_ref().unwrap();
self.new_table_info = Some(self.build_new_table_info(&table_info_value.table_info)?);
let new_table_info = AlterTableExecutor::validate_alter_table_expr(
&table_info_value.table_info,
self.data.task.alter_table.clone(),
)?;
self.new_table_info = Some(new_table_info);
// Safety: Checked in `AlterTableProcedure::new`.
let alter_kind = self.data.task.alter_table.kind.as_ref().unwrap();
@@ -140,9 +165,7 @@ impl AlterTableProcedure {
self.data.region_distribution =
Some(region_distribution(&physical_table_route.region_routes));
let leaders = find_leaders(&physical_table_route.region_routes);
let mut alter_region_tasks = Vec::with_capacity(leaders.len());
let alter_kind = self.make_region_alter_kind()?;
info!(
@@ -155,31 +178,14 @@ impl AlterTableProcedure {
ensure!(!leaders.is_empty(), NoLeaderSnafu { table_id });
// Puts the poison before submitting alter region requests to datanodes.
self.put_poison(ctx_provider, procedure_id).await?;
for datanode in leaders {
let requester = self.context.node_manager.datanode(&datanode).await;
let regions = find_leader_regions(&physical_table_route.region_routes, &datanode);
for region in regions {
let region_id = RegionId::new(table_id, region);
let request = self.make_alter_region_request(region_id, alter_kind.clone())?;
debug!("Submitting {request:?} to {datanode}");
let datanode = datanode.clone();
let requester = requester.clone();
alter_region_tasks.push(async move {
requester
.handle(request)
.await
.map_err(add_peer_context_if_needed(datanode))
});
}
}
let results = future::join_all(alter_region_tasks)
.await
.into_iter()
.collect::<Vec<_>>();
let results = self
.executor
.on_alter_regions(
&self.context.node_manager,
&physical_table_route.region_routes,
alter_kind,
)
.await;
match handle_multiple_results(results) {
MultipleResults::PartialRetryable(error) => {
@@ -202,9 +208,9 @@ impl AlterTableProcedure {
})
}
MultipleResults::Ok(results) => {
self.submit_sync_region_requests(results, &physical_table_route.region_routes)
self.submit_sync_region_requests(&results, &physical_table_route.region_routes)
.await;
self.data.state = AlterTableState::UpdateMetadata;
self.handle_alter_region_response(results)?;
Ok(Status::executing_with_clean_poisons(true))
}
MultipleResults::AllNonRetryable(error) => {
@@ -220,9 +226,22 @@ impl AlterTableProcedure {
}
}
fn handle_alter_region_response(&mut self, mut results: Vec<RegionResponse>) -> Result<()> {
self.data.state = AlterTableState::UpdateMetadata;
if let Some(column_metadatas) =
extract_column_metadatas(&mut results, TABLE_COLUMN_METADATA_EXTENSION_KEY)?
{
self.data.column_metadatas = column_metadatas;
} else {
warn!("altering table result doesn't contains extension key `{TABLE_COLUMN_METADATA_EXTENSION_KEY}`,leaving the table's column metadata unchanged");
}
Ok(())
}
async fn submit_sync_region_requests(
&mut self,
results: Vec<RegionResponse>,
results: &[RegionResponse],
region_routes: &[RegionRoute],
) {
// Safety: filled in `prepare` step.
@@ -244,39 +263,34 @@ impl AlterTableProcedure {
pub(crate) async fn on_update_metadata(&mut self) -> Result<Status> {
let table_id = self.data.table_id();
let table_ref = self.data.table_ref();
// Safety: checked before.
// Safety: filled in `fill_table_info`.
let table_info_value = self.data.table_info_value.as_ref().unwrap();
// Safety: Checked in `AlterTableProcedure::new`.
let alter_kind = self.data.task.alter_table.kind.as_ref().unwrap();
// Gets the table info from the cache or builds it.
let new_info = match &self.new_table_info {
let new_info = match &self.new_table_info {
Some(cached) => cached.clone(),
None => self.build_new_table_info(&table_info_value.table_info)
None => AlterTableExecutor::validate_alter_table_expr(
&table_info_value.table_info,
self.data.task.alter_table.clone(),
)
.inspect_err(|e| {
// We already check the table info in the prepare step so this should not happen.
error!(e; "Unable to build info for table {} in update metadata step, table_id: {}", table_ref, table_id);
})?,
};
debug!(
"Starting update table: {} metadata, new table info {:?}",
table_ref.to_string(),
new_info
);
// Safety: Checked in `AlterTableProcedure::new`.
let alter_kind = self.data.task.alter_table.kind.as_ref().unwrap();
if let Kind::RenameTable(RenameTable { new_table_name }) = alter_kind {
self.on_update_metadata_for_rename(new_table_name.to_string(), table_info_value)
.await?;
} else {
// region distribution is set in submit_alter_region_requests
let region_distribution = self.data.region_distribution.as_ref().unwrap().clone();
self.on_update_metadata_for_alter(
new_info.into(),
region_distribution,
// Safety: region distribution is set in `submit_alter_region_requests`.
self.executor
.on_alter_metadata(
&self.context.table_metadata_manager,
table_info_value,
self.data.region_distribution.as_ref(),
new_info.into(),
&self.data.column_metadatas,
)
.await?;
}
info!("Updated table metadata for table {table_ref}, table_id: {table_id}, kind: {alter_kind:?}");
self.data.state = AlterTableState::InvalidateTableCache;
@@ -285,18 +299,9 @@ impl AlterTableProcedure {
/// Broadcasts the invalidating table cache instructions.
async fn on_broadcast(&mut self) -> Result<Status> {
let cache_invalidator = &self.context.cache_invalidator;
cache_invalidator
.invalidate(
&Context::default(),
&[
CacheIdent::TableId(self.data.table_id()),
CacheIdent::TableName(self.data.table_ref().into()),
],
)
self.executor
.invalidate_table_cache(&self.context.cache_invalidator)
.await?;
Ok(Status::done())
}
@@ -318,6 +323,16 @@ impl AlterTableProcedure {
lock_key
}
#[cfg(test)]
pub(crate) fn data(&self) -> &AlterTableData {
&self.data
}
#[cfg(test)]
pub(crate) fn mut_data(&mut self) -> &mut AlterTableData {
&mut self.data
}
}
#[async_trait]
@@ -380,6 +395,8 @@ pub struct AlterTableData {
state: AlterTableState,
task: AlterTableTask,
table_id: TableId,
#[serde(default)]
column_metadatas: Vec<ColumnMetadata>,
/// Table info value before alteration.
table_info_value: Option<DeserializedValueWithBytes<TableInfoValue>>,
/// Region distribution for table in case we need to update region options.
@@ -392,6 +409,7 @@ impl AlterTableData {
state: AlterTableState::Prepare,
task,
table_id,
column_metadatas: vec![],
table_info_value: None,
region_distribution: None,
}
@@ -410,4 +428,14 @@ impl AlterTableData {
.as_ref()
.map(|value| &value.table_info)
}
#[cfg(test)]
pub(crate) fn column_metadatas(&self) -> &[ColumnMetadata] {
&self.column_metadatas
}
#[cfg(test)]
pub(crate) fn set_column_metadatas(&mut self, column_metadatas: Vec<ColumnMetadata>) {
self.column_metadatas = column_metadatas;
}
}

View File

@@ -1,62 +0,0 @@
// 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::alter_table_expr::Kind;
use api::v1::RenameTable;
use common_catalog::format_full_table_name;
use snafu::ensure;
use crate::ddl::alter_table::AlterTableProcedure;
use crate::error::{self, Result};
use crate::key::table_name::TableNameKey;
impl AlterTableProcedure {
/// Checks:
/// - The new table name doesn't exist (rename).
/// - Table exists.
pub(crate) async fn check_alter(&self) -> Result<()> {
let alter_expr = &self.data.task.alter_table;
let catalog = &alter_expr.catalog_name;
let schema = &alter_expr.schema_name;
let table_name = &alter_expr.table_name;
// Safety: Checked in `AlterTableProcedure::new`.
let alter_kind = self.data.task.alter_table.kind.as_ref().unwrap();
let manager = &self.context.table_metadata_manager;
if let Kind::RenameTable(RenameTable { new_table_name }) = alter_kind {
let new_table_name_key = TableNameKey::new(catalog, schema, new_table_name);
let exists = manager
.table_name_manager()
.exists(new_table_name_key)
.await?;
ensure!(
!exists,
error::TableAlreadyExistsSnafu {
table_name: format_full_table_name(catalog, schema, new_table_name),
}
)
}
let table_name_key = TableNameKey::new(catalog, schema, table_name);
let exists = manager.table_name_manager().exists(table_name_key).await?;
ensure!(
exists,
error::TableNotFoundSnafu {
table_name: format_full_table_name(catalog, schema, &alter_expr.table_name),
}
);
Ok(())
}
}

View File

@@ -0,0 +1,308 @@
// 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 api::region::RegionResponse;
use api::v1::region::region_request::Body;
use api::v1::region::{alter_request, AlterRequest, RegionRequest, RegionRequestHeader};
use api::v1::AlterTableExpr;
use common_catalog::format_full_table_name;
use common_grpc_expr::alter_expr_to_request;
use common_telemetry::debug;
use common_telemetry::tracing_context::TracingContext;
use futures::future;
use snafu::{ensure, ResultExt};
use store_api::metadata::ColumnMetadata;
use store_api::storage::{RegionId, TableId};
use table::metadata::{RawTableInfo, TableInfo};
use table::requests::AlterKind;
use table::table_name::TableName;
use crate::cache_invalidator::{CacheInvalidatorRef, Context};
use crate::ddl::utils::{add_peer_context_if_needed, raw_table_info};
use crate::error::{self, Result, UnexpectedSnafu};
use crate::instruction::CacheIdent;
use crate::key::table_info::TableInfoValue;
use crate::key::table_name::TableNameKey;
use crate::key::{DeserializedValueWithBytes, RegionDistribution, TableMetadataManagerRef};
use crate::node_manager::NodeManagerRef;
use crate::rpc::router::{find_leaders, region_distribution, RegionRoute};
/// [AlterTableExecutor] performs:
/// - Alters the metadata of the table.
/// - Alters regions on the datanode nodes.
pub struct AlterTableExecutor {
table: TableName,
table_id: TableId,
/// The new table name if the alter kind is rename table.
new_table_name: Option<String>,
}
impl AlterTableExecutor {
/// Creates a new [`AlterTableExecutor`].
pub fn new(table: TableName, table_id: TableId, new_table_name: Option<String>) -> Self {
Self {
table,
table_id,
new_table_name,
}
}
/// Prepares to alter the table.
///
/// ## Checks:
/// - The new table name doesn't exist (rename).
/// - Table exists.
pub(crate) async fn on_prepare(
&self,
table_metadata_manager: &TableMetadataManagerRef,
) -> Result<()> {
let catalog = &self.table.catalog_name;
let schema = &self.table.schema_name;
let table_name = &self.table.table_name;
let manager = table_metadata_manager;
if let Some(new_table_name) = &self.new_table_name {
let new_table_name_key = TableNameKey::new(catalog, schema, new_table_name);
let exists = manager
.table_name_manager()
.exists(new_table_name_key)
.await?;
ensure!(
!exists,
error::TableAlreadyExistsSnafu {
table_name: format_full_table_name(catalog, schema, new_table_name),
}
)
}
let table_name_key = TableNameKey::new(catalog, schema, table_name);
let exists = manager.table_name_manager().exists(table_name_key).await?;
ensure!(
exists,
error::TableNotFoundSnafu {
table_name: format_full_table_name(catalog, schema, table_name),
}
);
Ok(())
}
/// Validates the alter table expression and builds the new table info.
///
/// This validation is performed early to ensure the alteration is valid before
/// proceeding to the `on_alter_metadata` state, where regions have already been altered.
/// Building the new table info here allows us to catch any issues with the
/// alteration before committing metadata changes.
pub(crate) fn validate_alter_table_expr(
table_info: &RawTableInfo,
alter_table_expr: AlterTableExpr,
) -> Result<TableInfo> {
build_new_table_info(table_info, alter_table_expr)
}
/// Updates table metadata for alter table operation.
pub(crate) async fn on_alter_metadata(
&self,
table_metadata_manager: &TableMetadataManagerRef,
current_table_info_value: &DeserializedValueWithBytes<TableInfoValue>,
region_distribution: Option<&RegionDistribution>,
mut raw_table_info: RawTableInfo,
column_metadatas: &[ColumnMetadata],
) -> Result<()> {
let table_ref = self.table.table_ref();
let table_id = self.table_id;
if let Some(new_table_name) = &self.new_table_name {
debug!(
"Starting update table: {} metadata, table_id: {}, new table info: {:?}, new table name: {}",
table_ref, table_id, raw_table_info, new_table_name
);
table_metadata_manager
.rename_table(current_table_info_value, new_table_name.to_string())
.await?;
} else {
debug!(
"Starting update table: {} metadata, table_id: {}, new table info: {:?}",
table_ref, table_id, raw_table_info
);
ensure!(
region_distribution.is_some(),
UnexpectedSnafu {
err_msg: "region distribution is not set when updating table metadata",
}
);
if !column_metadatas.is_empty() {
raw_table_info::update_table_info_column_ids(&mut raw_table_info, column_metadatas);
}
table_metadata_manager
.update_table_info(
current_table_info_value,
region_distribution.cloned(),
raw_table_info,
)
.await?;
}
Ok(())
}
/// Alters regions on the datanode nodes.
pub(crate) async fn on_alter_regions(
&self,
node_manager: &NodeManagerRef,
region_routes: &[RegionRoute],
kind: Option<alter_request::Kind>,
) -> Vec<Result<RegionResponse>> {
let region_distribution = region_distribution(region_routes);
let leaders = find_leaders(region_routes)
.into_iter()
.map(|p| (p.id, p))
.collect::<HashMap<_, _>>();
let total_num_region = region_distribution
.values()
.map(|r| r.leader_regions.len())
.sum::<usize>();
let mut alter_region_tasks = Vec::with_capacity(total_num_region);
for (datanode_id, region_role_set) in region_distribution {
if region_role_set.leader_regions.is_empty() {
continue;
}
// Safety: must exists.
let peer = leaders.get(&datanode_id).unwrap();
let requester = node_manager.datanode(peer).await;
for region_id in region_role_set.leader_regions {
let region_id = RegionId::new(self.table_id, region_id);
let request = make_alter_region_request(region_id, kind.clone());
let requester = requester.clone();
let peer = peer.clone();
alter_region_tasks.push(async move {
requester
.handle(request)
.await
.map_err(add_peer_context_if_needed(peer))
});
}
}
future::join_all(alter_region_tasks)
.await
.into_iter()
.collect::<Vec<_>>()
}
/// Invalidates cache for the table.
pub(crate) async fn invalidate_table_cache(
&self,
cache_invalidator: &CacheInvalidatorRef,
) -> Result<()> {
let ctx = Context {
subject: Some(format!(
"Invalidate table cache by altering table {}, table_id: {}",
self.table.table_ref(),
self.table_id,
)),
};
cache_invalidator
.invalidate(
&ctx,
&[
CacheIdent::TableName(self.table.clone()),
CacheIdent::TableId(self.table_id),
],
)
.await?;
Ok(())
}
}
/// Makes alter region request.
pub(crate) fn make_alter_region_request(
region_id: RegionId,
kind: Option<alter_request::Kind>,
) -> RegionRequest {
RegionRequest {
header: Some(RegionRequestHeader {
tracing_context: TracingContext::from_current_span().to_w3c(),
..Default::default()
}),
body: Some(Body::Alter(AlterRequest {
region_id: region_id.as_u64(),
kind,
..Default::default()
})),
}
}
/// Builds new table info after alteration.
///
/// This function creates a new table info by applying the alter table expression
/// to the existing table info. For add column operations, it increments the
/// `next_column_id` by the number of columns being added, which may result in gaps
/// in the column id sequence.
fn build_new_table_info(
table_info: &RawTableInfo,
alter_table_expr: AlterTableExpr,
) -> Result<TableInfo> {
let table_info =
TableInfo::try_from(table_info.clone()).context(error::ConvertRawTableInfoSnafu)?;
let schema_name = &table_info.schema_name;
let catalog_name = &table_info.catalog_name;
let table_name = &table_info.name;
let table_id = table_info.ident.table_id;
let request = alter_expr_to_request(table_id, alter_table_expr)
.context(error::ConvertAlterTableRequestSnafu)?;
let new_meta = table_info
.meta
.builder_with_alter_kind(table_name, &request.alter_kind)
.context(error::TableSnafu)?
.build()
.with_context(|_| error::BuildTableMetaSnafu {
table_name: format_full_table_name(catalog_name, schema_name, table_name),
})?;
let mut new_info = table_info.clone();
new_info.meta = new_meta;
new_info.ident.version = table_info.ident.version + 1;
match request.alter_kind {
AlterKind::AddColumns { columns } => {
// Bumps the column id for the new columns.
// It may bump more than the actual number of columns added if there are
// existing columns, but it's fine.
new_info.meta.next_column_id += columns.len() as u32;
}
AlterKind::RenameTable { new_table_name } => {
new_info.name = new_table_name.to_string();
}
AlterKind::DropColumns { .. }
| AlterKind::ModifyColumnTypes { .. }
| AlterKind::SetTableOptions { .. }
| AlterKind::UnsetTableOptions { .. }
| AlterKind::SetIndex { .. }
| AlterKind::UnsetIndex { .. }
| AlterKind::DropDefaults { .. } => {}
}
Ok(new_info)
}

View File

@@ -15,43 +15,16 @@
use std::collections::HashSet;
use api::v1::alter_table_expr::Kind;
use api::v1::region::region_request::Body;
use api::v1::region::{
alter_request, AddColumn, AddColumns, AlterRequest, DropColumn, DropColumns, RegionColumnDef,
RegionRequest, RegionRequestHeader,
alter_request, AddColumn, AddColumns, DropColumn, DropColumns, RegionColumnDef,
};
use common_telemetry::tracing_context::TracingContext;
use snafu::OptionExt;
use store_api::storage::RegionId;
use table::metadata::RawTableInfo;
use crate::ddl::alter_table::AlterTableProcedure;
use crate::error::{InvalidProtoMsgSnafu, Result};
impl AlterTableProcedure {
/// Makes alter region request from existing an alter kind.
/// Region alter request always add columns if not exist.
pub(crate) fn make_alter_region_request(
&self,
region_id: RegionId,
kind: Option<alter_request::Kind>,
) -> Result<RegionRequest> {
// Safety: checked
let table_info = self.data.table_info().unwrap();
Ok(RegionRequest {
header: Some(RegionRequestHeader {
tracing_context: TracingContext::from_current_span().to_w3c(),
..Default::default()
}),
body: Some(Body::Alter(AlterRequest {
region_id: region_id.as_u64(),
schema_version: table_info.ident.version,
kind,
})),
})
}
/// Makes alter kind proto that all regions can reuse.
/// Region alter request always add columns if not exist.
pub(crate) fn make_region_alter_kind(&self) -> Result<Option<alter_request::Kind>> {
@@ -155,6 +128,7 @@ mod tests {
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
use store_api::storage::{RegionId, TableId};
use crate::ddl::alter_table::executor::make_alter_region_request;
use crate::ddl::alter_table::AlterTableProcedure;
use crate::ddl::test_util::columns::TestColumnDefBuilder;
use crate::ddl::test_util::create_table::{
@@ -261,15 +235,13 @@ mod tests {
let mut procedure = AlterTableProcedure::new(table_id, task, ddl_context).unwrap();
procedure.on_prepare().await.unwrap();
let alter_kind = procedure.make_region_alter_kind().unwrap();
let Some(Body::Alter(alter_region_request)) = procedure
.make_alter_region_request(region_id, alter_kind)
.unwrap()
.body
let Some(Body::Alter(alter_region_request)) =
make_alter_region_request(region_id, alter_kind).body
else {
unreachable!()
};
assert_eq!(alter_region_request.region_id, region_id.as_u64());
assert_eq!(alter_region_request.schema_version, 1);
assert_eq!(alter_region_request.schema_version, 0);
assert_eq!(
alter_region_request.kind,
Some(region::alter_request::Kind::AddColumns(
@@ -319,15 +291,13 @@ mod tests {
let mut procedure = AlterTableProcedure::new(table_id, task, ddl_context).unwrap();
procedure.on_prepare().await.unwrap();
let alter_kind = procedure.make_region_alter_kind().unwrap();
let Some(Body::Alter(alter_region_request)) = procedure
.make_alter_region_request(region_id, alter_kind)
.unwrap()
.body
let Some(Body::Alter(alter_region_request)) =
make_alter_region_request(region_id, alter_kind).body
else {
unreachable!()
};
assert_eq!(alter_region_request.region_id, region_id.as_u64());
assert_eq!(alter_region_request.schema_version, 1);
assert_eq!(alter_region_request.schema_version, 0);
assert_eq!(
alter_region_request.kind,
Some(region::alter_request::Kind::ModifyColumnTypes(

View File

@@ -1,103 +0,0 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use common_grpc_expr::alter_expr_to_request;
use snafu::ResultExt;
use table::metadata::{RawTableInfo, TableInfo};
use table::requests::AlterKind;
use crate::ddl::alter_table::AlterTableProcedure;
use crate::error::{self, Result};
use crate::key::table_info::TableInfoValue;
use crate::key::{DeserializedValueWithBytes, RegionDistribution};
impl AlterTableProcedure {
/// Builds new table info after alteration.
/// It bumps the column id of the table by the number of the add column requests.
/// So there may be holes in the column id sequence.
pub(crate) fn build_new_table_info(&self, table_info: &RawTableInfo) -> Result<TableInfo> {
let table_info =
TableInfo::try_from(table_info.clone()).context(error::ConvertRawTableInfoSnafu)?;
let table_ref = self.data.table_ref();
let alter_expr = self.data.task.alter_table.clone();
let request = alter_expr_to_request(self.data.table_id(), alter_expr)
.context(error::ConvertAlterTableRequestSnafu)?;
let new_meta = table_info
.meta
.builder_with_alter_kind(table_ref.table, &request.alter_kind)
.context(error::TableSnafu)?
.build()
.with_context(|_| error::BuildTableMetaSnafu {
table_name: table_ref.table,
})?;
let mut new_info = table_info.clone();
new_info.meta = new_meta;
new_info.ident.version = table_info.ident.version + 1;
match request.alter_kind {
AlterKind::AddColumns { columns } => {
// Bumps the column id for the new columns.
// It may bump more than the actual number of columns added if there are
// existing columns, but it's fine.
new_info.meta.next_column_id += columns.len() as u32;
}
AlterKind::RenameTable { new_table_name } => {
new_info.name = new_table_name.to_string();
}
AlterKind::DropColumns { .. }
| AlterKind::ModifyColumnTypes { .. }
| AlterKind::SetTableOptions { .. }
| AlterKind::UnsetTableOptions { .. }
| AlterKind::SetIndex { .. }
| AlterKind::UnsetIndex { .. }
| AlterKind::DropDefaults { .. } => {}
}
Ok(new_info)
}
/// Updates table metadata for rename table operation.
pub(crate) async fn on_update_metadata_for_rename(
&self,
new_table_name: String,
current_table_info_value: &DeserializedValueWithBytes<TableInfoValue>,
) -> Result<()> {
let table_metadata_manager = &self.context.table_metadata_manager;
table_metadata_manager
.rename_table(current_table_info_value, new_table_name)
.await?;
Ok(())
}
/// Updates table metadata for alter table operation.
pub(crate) async fn on_update_metadata_for_alter(
&self,
new_table_info: RawTableInfo,
region_distribution: RegionDistribution,
current_table_info_value: &DeserializedValueWithBytes<TableInfoValue>,
) -> Result<()> {
let table_metadata_manager = &self.context.table_metadata_manager;
table_metadata_manager
.update_table_info(
current_table_info_value,
Some(region_distribution),
new_table_info,
)
.await?;
Ok(())
}
}

View File

@@ -167,6 +167,25 @@ impl CreateFlowProcedure {
}
self.collect_source_tables().await?;
// Validate that source and sink tables are not the same
let sink_table_name = &self.data.task.sink_table_name;
if self
.data
.task
.source_table_names
.iter()
.any(|source| source == sink_table_name)
{
return error::UnsupportedSnafu {
operation: format!(
"Creating flow with source and sink table being the same: {}",
sink_table_name
),
}
.fail();
}
if self.data.flow_id.is_none() {
self.allocate_flow_id().await?;
}

View File

@@ -27,7 +27,7 @@ use common_telemetry::{debug, error, warn};
use futures::future;
pub use region_request::create_region_request_builder;
use serde::{Deserialize, Serialize};
use snafu::{ensure, ResultExt};
use snafu::ResultExt;
use store_api::metadata::ColumnMetadata;
use store_api::metric_engine_consts::ALTER_PHYSICAL_EXTENSION_KEY;
use store_api::storage::{RegionId, RegionNumber};
@@ -35,10 +35,11 @@ use strum::AsRefStr;
use table::metadata::{RawTableInfo, TableId};
use crate::ddl::utils::{
add_peer_context_if_needed, map_to_procedure_error, sync_follower_regions,
add_peer_context_if_needed, extract_column_metadatas, map_to_procedure_error,
sync_follower_regions,
};
use crate::ddl::DdlContext;
use crate::error::{DecodeJsonSnafu, MetadataCorruptionSnafu, Result};
use crate::error::Result;
use crate::key::table_route::TableRouteValue;
use crate::lock_key::{CatalogLock, SchemaLock, TableLock, TableNameLock};
use crate::metrics;
@@ -166,47 +167,23 @@ impl CreateLogicalTablesProcedure {
.into_iter()
.collect::<Result<Vec<_>>>()?;
// Collects response from datanodes.
let phy_raw_schemas = results
.iter_mut()
.map(|res| res.extensions.remove(ALTER_PHYSICAL_EXTENSION_KEY))
.collect::<Vec<_>>();
if phy_raw_schemas.is_empty() {
self.submit_sync_region_requests(results, region_routes)
.await;
self.data.state = CreateTablesState::CreateMetadata;
return Ok(Status::executing(false));
}
// Verify all the physical schemas are the same
// Safety: previous check ensures this vec is not empty
let first = phy_raw_schemas.first().unwrap();
ensure!(
phy_raw_schemas.iter().all(|x| x == first),
MetadataCorruptionSnafu {
err_msg: "The physical schemas from datanodes are not the same."
}
);
// Decodes the physical raw schemas
if let Some(phy_raw_schemas) = first {
self.data.physical_columns =
ColumnMetadata::decode_list(phy_raw_schemas).context(DecodeJsonSnafu)?;
if let Some(column_metadatas) =
extract_column_metadatas(&mut results, ALTER_PHYSICAL_EXTENSION_KEY)?
{
self.data.physical_columns = column_metadatas;
} else {
warn!("creating logical table result doesn't contains extension key `{ALTER_PHYSICAL_EXTENSION_KEY}`,leaving the physical table's schema unchanged");
}
self.submit_sync_region_requests(results, region_routes)
self.submit_sync_region_requests(&results, region_routes)
.await;
self.data.state = CreateTablesState::CreateMetadata;
Ok(Status::executing(true))
}
async fn submit_sync_region_requests(
&self,
results: Vec<RegionResponse>,
results: &[RegionResponse],
region_routes: &[RegionRoute],
) {
if let Err(err) = sync_follower_regions(

View File

@@ -22,7 +22,7 @@ use table::table_name::TableName;
use crate::cache_invalidator::Context;
use crate::ddl::create_logical_tables::CreateLogicalTablesProcedure;
use crate::ddl::physical_table_metadata;
use crate::ddl::utils::raw_table_info;
use crate::error::{Result, TableInfoNotFoundSnafu};
use crate::instruction::CacheIdent;
@@ -47,7 +47,7 @@ impl CreateLogicalTablesProcedure {
// Generates new table info
let raw_table_info = physical_table_info.deref().table_info.clone();
let new_table_info = physical_table_metadata::build_new_physical_table_info(
let new_table_info = raw_table_info::build_new_physical_table_info(
raw_table_info,
&self.data.physical_columns,
);

View File

@@ -22,20 +22,23 @@ use common_procedure::error::{
ExternalSnafu, FromJsonSnafu, Result as ProcedureResult, ToJsonSnafu,
};
use common_procedure::{Context as ProcedureContext, LockKey, Procedure, Status};
use common_telemetry::info;
use common_telemetry::tracing_context::TracingContext;
use common_telemetry::{info, warn};
use futures::future::join_all;
use serde::{Deserialize, Serialize};
use snafu::{ensure, OptionExt, ResultExt};
use store_api::metadata::ColumnMetadata;
use store_api::metric_engine_consts::TABLE_COLUMN_METADATA_EXTENSION_KEY;
use store_api::storage::{RegionId, RegionNumber};
use strum::AsRefStr;
use table::metadata::{RawTableInfo, TableId};
use table::table_reference::TableReference;
use crate::ddl::create_table_template::{build_template, CreateRequestBuilder};
use crate::ddl::utils::raw_table_info::update_table_info_column_ids;
use crate::ddl::utils::{
add_peer_context_if_needed, convert_region_routes_to_detecting_regions, map_to_procedure_error,
region_storage_path,
add_peer_context_if_needed, convert_region_routes_to_detecting_regions,
extract_column_metadatas, map_to_procedure_error, region_storage_path,
};
use crate::ddl::{DdlContext, TableMetadata};
use crate::error::{self, Result};
@@ -243,14 +246,21 @@ impl CreateTableProcedure {
}
}
join_all(create_region_tasks)
self.creator.data.state = CreateTableState::CreateMetadata;
let mut results = join_all(create_region_tasks)
.await
.into_iter()
.collect::<Result<Vec<_>>>()?;
self.creator.data.state = CreateTableState::CreateMetadata;
if let Some(column_metadatas) =
extract_column_metadatas(&mut results, TABLE_COLUMN_METADATA_EXTENSION_KEY)?
{
self.creator.data.column_metadatas = column_metadatas;
} else {
warn!("creating table result doesn't contains extension key `{TABLE_COLUMN_METADATA_EXTENSION_KEY}`,leaving the table's column metadata unchanged");
}
// TODO(weny): Add more tests.
Ok(Status::executing(true))
}
@@ -262,7 +272,10 @@ impl CreateTableProcedure {
let table_id = self.table_id();
let manager = &self.context.table_metadata_manager;
let raw_table_info = self.table_info().clone();
let mut raw_table_info = self.table_info().clone();
if !self.creator.data.column_metadatas.is_empty() {
update_table_info_column_ids(&mut raw_table_info, &self.creator.data.column_metadatas);
}
// Safety: the region_wal_options must be allocated.
let region_wal_options = self.region_wal_options()?.clone();
// Safety: the table_route must be allocated.
@@ -346,6 +359,7 @@ impl TableCreator {
Self {
data: CreateTableData {
state: CreateTableState::Prepare,
column_metadatas: vec![],
task,
table_route: None,
region_wal_options: None,
@@ -407,6 +421,8 @@ pub enum CreateTableState {
pub struct CreateTableData {
pub state: CreateTableState,
pub task: CreateTableTask,
#[serde(default)]
pub column_metadatas: Vec<ColumnMetadata>,
/// None stands for not allocated yet.
table_route: Option<PhysicalTableRouteValue>,
/// None stands for not allocated yet.

View File

@@ -185,11 +185,15 @@ impl DropTableExecutor {
.await
}
/// Invalidates frontend caches
/// Invalidates caches for the table.
pub async fn invalidate_table_cache(&self, ctx: &DdlContext) -> Result<()> {
let cache_invalidator = &ctx.cache_invalidator;
let ctx = Context {
subject: Some("Invalidate table cache by dropping table".to_string()),
subject: Some(format!(
"Invalidate table cache by dropping table {}, table_id: {}",
self.table.table_ref(),
self.table_id,
)),
};
cache_invalidator

View File

@@ -122,6 +122,7 @@ impl TableMetadataAllocator {
);
let peers = self.peer_allocator.alloc(regions).await?;
debug!("Allocated peers {:?} for table {}", peers, table_id);
let region_routes = task
.partitions
.iter()

View File

@@ -24,7 +24,14 @@ use std::collections::HashMap;
use api::v1::meta::Partition;
use api::v1::{ColumnDataType, SemanticType};
use common_procedure::Status;
use store_api::metric_engine_consts::{LOGICAL_TABLE_METADATA_KEY, METRIC_ENGINE_NAME};
use datatypes::prelude::ConcreteDataType;
use datatypes::schema::ColumnSchema;
use store_api::metadata::ColumnMetadata;
use store_api::metric_engine_consts::{
DATA_SCHEMA_TABLE_ID_COLUMN_NAME, DATA_SCHEMA_TSID_COLUMN_NAME, LOGICAL_TABLE_METADATA_KEY,
METRIC_ENGINE_NAME,
};
use store_api::storage::consts::ReservedColumnId;
use table::metadata::{RawTableInfo, TableId};
use crate::ddl::create_logical_tables::CreateLogicalTablesProcedure;
@@ -146,6 +153,7 @@ pub fn test_create_logical_table_task(name: &str) -> CreateTableTask {
}
}
/// Creates a physical table task with a single region.
pub fn test_create_physical_table_task(name: &str) -> CreateTableTask {
let create_table = TestCreateTableExprBuilder::default()
.column_defs([
@@ -182,3 +190,95 @@ pub fn test_create_physical_table_task(name: &str) -> CreateTableTask {
table_info,
}
}
/// Creates a column metadata list with tag fields.
pub fn test_column_metadatas(tag_fields: &[&str]) -> Vec<ColumnMetadata> {
let mut output = Vec::with_capacity(tag_fields.len() + 4);
output.extend([
ColumnMetadata {
column_schema: ColumnSchema::new(
"ts",
ConcreteDataType::timestamp_millisecond_datatype(),
false,
),
semantic_type: SemanticType::Timestamp,
column_id: 0,
},
ColumnMetadata {
column_schema: ColumnSchema::new("value", ConcreteDataType::float64_datatype(), false),
semantic_type: SemanticType::Field,
column_id: 1,
},
ColumnMetadata {
column_schema: ColumnSchema::new(
DATA_SCHEMA_TABLE_ID_COLUMN_NAME,
ConcreteDataType::timestamp_millisecond_datatype(),
false,
),
semantic_type: SemanticType::Tag,
column_id: ReservedColumnId::table_id(),
},
ColumnMetadata {
column_schema: ColumnSchema::new(
DATA_SCHEMA_TSID_COLUMN_NAME,
ConcreteDataType::float64_datatype(),
false,
),
semantic_type: SemanticType::Tag,
column_id: ReservedColumnId::tsid(),
},
]);
for (i, name) in tag_fields.iter().enumerate() {
output.push(ColumnMetadata {
column_schema: ColumnSchema::new(
name.to_string(),
ConcreteDataType::string_datatype(),
true,
),
semantic_type: SemanticType::Tag,
column_id: (i + 2) as u32,
});
}
output
}
/// Asserts the column names.
pub fn assert_column_name(table_info: &RawTableInfo, expected_column_names: &[&str]) {
assert_eq!(
table_info
.meta
.schema
.column_schemas
.iter()
.map(|c| c.name.to_string())
.collect::<Vec<_>>(),
expected_column_names
);
}
/// Asserts the column metadatas
pub fn assert_column_name_and_id(column_metadatas: &[ColumnMetadata], expected: &[(&str, u32)]) {
assert_eq!(expected.len(), column_metadatas.len());
for (name, id) in expected {
let column_metadata = column_metadatas
.iter()
.find(|c| c.column_id == *id)
.unwrap();
assert_eq!(column_metadata.column_schema.name, *name);
}
}
/// Gets the raw table info.
pub async fn get_raw_table_info(ddl_context: &DdlContext, table_id: TableId) -> RawTableInfo {
ddl_context
.table_metadata_manager
.table_info_manager()
.get(table_id)
.await
.unwrap()
.unwrap()
.into_inner()
.table_info
}

View File

@@ -132,6 +132,7 @@ pub fn build_raw_table_info_from_expr(expr: &CreateTableExpr) -> RawTableInfo {
options: TableOptions::try_from_iter(&expr.table_options).unwrap(),
created_on: DateTime::default(),
partition_key_indices: vec![],
column_ids: vec![],
},
table_type: TableType::Base,
}

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 api::region::RegionResponse;
use api::v1::region::RegionRequest;
use common_error::ext::{BoxedError, ErrorExt, StackError};
@@ -45,10 +47,13 @@ impl MockDatanodeHandler for () {
}
}
type RegionRequestHandler =
Arc<dyn Fn(Peer, RegionRequest) -> Result<RegionResponse> + Send + Sync>;
#[derive(Clone)]
pub struct DatanodeWatcher {
sender: mpsc::Sender<(Peer, RegionRequest)>,
handler: Option<fn(Peer, RegionRequest) -> Result<RegionResponse>>,
handler: Option<RegionRequestHandler>,
}
impl DatanodeWatcher {
@@ -61,9 +66,9 @@ impl DatanodeWatcher {
pub fn with_handler(
mut self,
user_handler: fn(Peer, RegionRequest) -> Result<RegionResponse>,
user_handler: impl Fn(Peer, RegionRequest) -> Result<RegionResponse> + Send + Sync + 'static,
) -> Self {
self.handler = Some(user_handler);
self.handler = Some(Arc::new(user_handler));
self
}
}
@@ -76,7 +81,7 @@ impl MockDatanodeHandler for DatanodeWatcher {
.send((peer.clone(), request.clone()))
.await
.unwrap();
if let Some(handler) = self.handler {
if let Some(handler) = self.handler.as_ref() {
handler(peer.clone(), request)
} else {
Ok(RegionResponse::new(0))

View File

@@ -23,17 +23,20 @@ use api::v1::{ColumnDataType, SemanticType};
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
use common_procedure::{Procedure, ProcedureId, Status};
use common_procedure_test::MockContextProvider;
use store_api::metric_engine_consts::MANIFEST_INFO_EXTENSION_KEY;
use store_api::metadata::ColumnMetadata;
use store_api::metric_engine_consts::{ALTER_PHYSICAL_EXTENSION_KEY, MANIFEST_INFO_EXTENSION_KEY};
use store_api::region_engine::RegionManifestInfo;
use store_api::storage::consts::ReservedColumnId;
use store_api::storage::RegionId;
use tokio::sync::mpsc;
use crate::ddl::alter_logical_tables::AlterLogicalTablesProcedure;
use crate::ddl::test_util::alter_table::TestAlterTableExprBuilder;
use crate::ddl::test_util::columns::TestColumnDefBuilder;
use crate::ddl::test_util::datanode_handler::{DatanodeWatcher, NaiveDatanodeHandler};
use crate::ddl::test_util::datanode_handler::DatanodeWatcher;
use crate::ddl::test_util::{
create_logical_table, create_physical_table, create_physical_table_metadata,
assert_column_name, create_logical_table, create_physical_table,
create_physical_table_metadata, get_raw_table_info, test_column_metadatas,
test_create_physical_table_task,
};
use crate::error::Error::{AlterLogicalTablesInvalidArguments, TableNotFound};
@@ -96,6 +99,52 @@ fn make_alter_logical_table_rename_task(
}
}
fn make_alters_request_handler(
column_metadatas: Vec<ColumnMetadata>,
) -> impl Fn(Peer, RegionRequest) -> Result<RegionResponse> {
move |_peer: Peer, request: RegionRequest| {
if let region_request::Body::Alters(_) = request.body.unwrap() {
let mut response = RegionResponse::new(0);
// Default region id for physical table.
let region_id = RegionId::new(1000, 1);
response.extensions.insert(
MANIFEST_INFO_EXTENSION_KEY.to_string(),
RegionManifestInfo::encode_list(&[(
region_id,
RegionManifestInfo::metric(1, 0, 2, 0),
)])
.unwrap(),
);
response.extensions.insert(
ALTER_PHYSICAL_EXTENSION_KEY.to_string(),
ColumnMetadata::encode_list(&column_metadatas).unwrap(),
);
return Ok(response);
}
Ok(RegionResponse::new(0))
}
}
fn assert_alters_request(
peer: Peer,
request: RegionRequest,
expected_peer_id: u64,
expected_region_ids: &[RegionId],
) {
assert_eq!(peer.id, expected_peer_id,);
let Some(region_request::Body::Alters(req)) = request.body else {
unreachable!();
};
for (i, region_id) in expected_region_ids.iter().enumerate() {
assert_eq!(
req.requests[i].region_id,
*region_id,
"actual region id: {}",
RegionId::from_u64(req.requests[i].region_id)
);
}
}
#[tokio::test]
async fn test_on_prepare_check_schema() {
let node_manager = Arc::new(MockDatanodeManager::new(()));
@@ -205,15 +254,20 @@ async fn test_on_prepare() {
#[tokio::test]
async fn test_on_update_metadata() {
let node_manager = Arc::new(MockDatanodeManager::new(NaiveDatanodeHandler));
common_telemetry::init_default_ut_logging();
let (tx, mut rx) = mpsc::channel(8);
let test_column_metadatas = test_column_metadatas(&["new_col", "mew_col"]);
let datanode_handler =
DatanodeWatcher::new(tx).with_handler(make_alters_request_handler(test_column_metadatas));
let node_manager = Arc::new(MockDatanodeManager::new(datanode_handler));
let ddl_context = new_ddl_context(node_manager);
// Creates physical table
let phy_id = create_physical_table(&ddl_context, "phy").await;
// Creates 3 logical tables
create_logical_table(ddl_context.clone(), phy_id, "table1").await;
create_logical_table(ddl_context.clone(), phy_id, "table2").await;
create_logical_table(ddl_context.clone(), phy_id, "table3").await;
let logical_table1_id = create_logical_table(ddl_context.clone(), phy_id, "table1").await;
let logical_table2_id = create_logical_table(ddl_context.clone(), phy_id, "table2").await;
let logical_table3_id = create_logical_table(ddl_context.clone(), phy_id, "table3").await;
create_logical_table(ddl_context.clone(), phy_id, "table4").await;
create_logical_table(ddl_context.clone(), phy_id, "table5").await;
@@ -223,7 +277,7 @@ async fn test_on_update_metadata() {
make_alter_logical_table_add_column_task(None, "table3", vec!["new_col".to_string()]),
];
let mut procedure = AlterLogicalTablesProcedure::new(tasks, phy_id, ddl_context);
let mut procedure = AlterLogicalTablesProcedure::new(tasks, phy_id, ddl_context.clone());
let mut status = procedure.on_prepare().await.unwrap();
assert_matches!(
status,
@@ -255,18 +309,52 @@ async fn test_on_update_metadata() {
clean_poisons: false
}
);
let (peer, request) = rx.try_recv().unwrap();
rx.try_recv().unwrap_err();
assert_alters_request(
peer,
request,
0,
&[
RegionId::new(logical_table1_id, 0),
RegionId::new(logical_table2_id, 0),
RegionId::new(logical_table3_id, 0),
],
);
let table_info = get_raw_table_info(&ddl_context, phy_id).await;
assert_column_name(
&table_info,
&["ts", "value", "__table_id", "__tsid", "new_col", "mew_col"],
);
assert_eq!(
table_info.meta.column_ids,
vec![
0,
1,
ReservedColumnId::table_id(),
ReservedColumnId::tsid(),
2,
3
]
);
}
#[tokio::test]
async fn test_on_part_duplicate_alter_request() {
let node_manager = Arc::new(MockDatanodeManager::new(NaiveDatanodeHandler));
let ddl_context = new_ddl_context(node_manager);
common_telemetry::init_default_ut_logging();
let (tx, mut rx) = mpsc::channel(8);
let column_metadatas = test_column_metadatas(&["col_0"]);
let handler =
DatanodeWatcher::new(tx).with_handler(make_alters_request_handler(column_metadatas));
let node_manager = Arc::new(MockDatanodeManager::new(handler));
let mut ddl_context = new_ddl_context(node_manager);
// Creates physical table
let phy_id = create_physical_table(&ddl_context, "phy").await;
// Creates 3 logical tables
create_logical_table(ddl_context.clone(), phy_id, "table1").await;
create_logical_table(ddl_context.clone(), phy_id, "table2").await;
let logical_table1_id = create_logical_table(ddl_context.clone(), phy_id, "table1").await;
let logical_table2_id = create_logical_table(ddl_context.clone(), phy_id, "table2").await;
let tasks = vec![
make_alter_logical_table_add_column_task(None, "table1", vec!["col_0".to_string()]),
@@ -305,6 +393,40 @@ async fn test_on_part_duplicate_alter_request() {
clean_poisons: false
}
);
let (peer, request) = rx.try_recv().unwrap();
rx.try_recv().unwrap_err();
assert_alters_request(
peer,
request,
0,
&[
RegionId::new(logical_table1_id, 0),
RegionId::new(logical_table2_id, 0),
],
);
let table_info = get_raw_table_info(&ddl_context, phy_id).await;
assert_column_name(
&table_info,
&["ts", "value", "__table_id", "__tsid", "col_0"],
);
assert_eq!(
table_info.meta.column_ids,
vec![
0,
1,
ReservedColumnId::table_id(),
ReservedColumnId::tsid(),
2
]
);
let (tx, mut rx) = mpsc::channel(8);
let column_metadatas = test_column_metadatas(&["col_0", "new_col_1", "new_col_2"]);
let handler =
DatanodeWatcher::new(tx).with_handler(make_alters_request_handler(column_metadatas));
let node_manager = Arc::new(MockDatanodeManager::new(handler));
ddl_context.node_manager = node_manager;
// re-alter
let tasks = vec![
@@ -357,6 +479,44 @@ async fn test_on_part_duplicate_alter_request() {
}
);
let (peer, request) = rx.try_recv().unwrap();
rx.try_recv().unwrap_err();
assert_alters_request(
peer,
request,
0,
&[
RegionId::new(logical_table1_id, 0),
RegionId::new(logical_table2_id, 0),
],
);
let table_info = get_raw_table_info(&ddl_context, phy_id).await;
assert_column_name(
&table_info,
&[
"ts",
"value",
"__table_id",
"__tsid",
"col_0",
"new_col_1",
"new_col_2",
],
);
assert_eq!(
table_info.meta.column_ids,
vec![
0,
1,
ReservedColumnId::table_id(),
ReservedColumnId::tsid(),
2,
3,
4,
]
);
let table_name_keys = vec![
TableNameKey::new(DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, "table1"),
TableNameKey::new(DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, "table2"),
@@ -422,27 +582,13 @@ async fn test_on_part_duplicate_alter_request() {
);
}
fn alters_request_handler(_peer: Peer, request: RegionRequest) -> Result<RegionResponse> {
if let region_request::Body::Alters(_) = request.body.unwrap() {
let mut response = RegionResponse::new(0);
// Default region id for physical table.
let region_id = RegionId::new(1000, 1);
response.extensions.insert(
MANIFEST_INFO_EXTENSION_KEY.to_string(),
RegionManifestInfo::encode_list(&[(region_id, RegionManifestInfo::metric(1, 0, 2, 0))])
.unwrap(),
);
return Ok(response);
}
Ok(RegionResponse::new(0))
}
#[tokio::test]
async fn test_on_submit_alter_region_request() {
common_telemetry::init_default_ut_logging();
let (tx, mut rx) = mpsc::channel(8);
let handler = DatanodeWatcher::new(tx).with_handler(alters_request_handler);
let column_metadatas = test_column_metadatas(&["new_col", "mew_col"]);
let handler =
DatanodeWatcher::new(tx).with_handler(make_alters_request_handler(column_metadatas));
let node_manager = Arc::new(MockDatanodeManager::new(handler));
let ddl_context = new_ddl_context(node_manager);

View File

@@ -30,7 +30,12 @@ use common_error::status_code::StatusCode;
use common_procedure::store::poison_store::PoisonStore;
use common_procedure::{ProcedureId, Status};
use common_procedure_test::MockContextProvider;
use store_api::metric_engine_consts::MANIFEST_INFO_EXTENSION_KEY;
use datatypes::prelude::ConcreteDataType;
use datatypes::schema::ColumnSchema;
use store_api::metadata::ColumnMetadata;
use store_api::metric_engine_consts::{
MANIFEST_INFO_EXTENSION_KEY, TABLE_COLUMN_METADATA_EXTENSION_KEY,
};
use store_api::region_engine::RegionManifestInfo;
use store_api::storage::RegionId;
use table::requests::TTL_KEY;
@@ -43,6 +48,7 @@ use crate::ddl::test_util::datanode_handler::{
AllFailureDatanodeHandler, DatanodeWatcher, PartialSuccessDatanodeHandler,
RequestOutdatedErrorDatanodeHandler,
};
use crate::ddl::test_util::{assert_column_name, assert_column_name_and_id};
use crate::error::{Error, Result};
use crate::key::datanode_table::DatanodeTableKey;
use crate::key::table_name::TableNameKey;
@@ -179,6 +185,30 @@ fn alter_request_handler(_peer: Peer, request: RegionRequest) -> Result<RegionRe
RegionManifestInfo::encode_list(&[(region_id, RegionManifestInfo::mito(1, 1))])
.unwrap(),
);
response.extensions.insert(
TABLE_COLUMN_METADATA_EXTENSION_KEY.to_string(),
ColumnMetadata::encode_list(&[
ColumnMetadata {
column_schema: ColumnSchema::new(
"ts",
ConcreteDataType::timestamp_millisecond_datatype(),
false,
),
semantic_type: SemanticType::Timestamp,
column_id: 0,
},
ColumnMetadata {
column_schema: ColumnSchema::new(
"host",
ConcreteDataType::float64_datatype(),
false,
),
semantic_type: SemanticType::Tag,
column_id: 1,
},
])
.unwrap(),
);
return Ok(response);
}
@@ -187,6 +217,7 @@ fn alter_request_handler(_peer: Peer, request: RegionRequest) -> Result<RegionRe
#[tokio::test]
async fn test_on_submit_alter_request() {
common_telemetry::init_default_ut_logging();
let (tx, mut rx) = mpsc::channel(8);
let datanode_handler = DatanodeWatcher::new(tx).with_handler(alter_request_handler);
let node_manager = Arc::new(MockDatanodeManager::new(datanode_handler));
@@ -234,6 +265,8 @@ async fn test_on_submit_alter_request() {
assert_sync_request(peer, request, 4, RegionId::new(table_id, 2), 1);
let (peer, request) = results.remove(0);
assert_sync_request(peer, request, 5, RegionId::new(table_id, 1), 1);
let column_metadatas = procedure.data().column_metadatas();
assert_column_name_and_id(column_metadatas, &[("ts", 0), ("host", 1)]);
}
#[tokio::test]
@@ -378,6 +411,7 @@ async fn test_on_update_metadata_rename() {
#[tokio::test]
async fn test_on_update_metadata_add_columns() {
common_telemetry::init_default_ut_logging();
let node_manager = Arc::new(MockDatanodeManager::new(()));
let ddl_context = new_ddl_context(node_manager);
let table_name = "foo";
@@ -431,6 +465,34 @@ async fn test_on_update_metadata_add_columns() {
.submit_alter_region_requests(procedure_id, provider.as_ref())
.await
.unwrap();
// Returned column metadatas is empty.
assert!(procedure.data().column_metadatas().is_empty());
procedure.mut_data().set_column_metadatas(vec![
ColumnMetadata {
column_schema: ColumnSchema::new(
"ts",
ConcreteDataType::timestamp_millisecond_datatype(),
false,
),
semantic_type: SemanticType::Timestamp,
column_id: 0,
},
ColumnMetadata {
column_schema: ColumnSchema::new("host", ConcreteDataType::float64_datatype(), false),
semantic_type: SemanticType::Tag,
column_id: 1,
},
ColumnMetadata {
column_schema: ColumnSchema::new("cpu", ConcreteDataType::float64_datatype(), false),
semantic_type: SemanticType::Tag,
column_id: 2,
},
ColumnMetadata {
column_schema: ColumnSchema::new("my_tag3", ConcreteDataType::string_datatype(), true),
semantic_type: SemanticType::Tag,
column_id: 3,
},
]);
procedure.on_update_metadata().await.unwrap();
let table_info = ddl_context
@@ -447,6 +509,8 @@ async fn test_on_update_metadata_add_columns() {
table_info.meta.schema.column_schemas.len() as u32,
table_info.meta.next_column_id
);
assert_column_name(&table_info, &["ts", "host", "cpu", "my_tag3"]);
assert_eq!(table_info.meta.column_ids, vec![0, 1, 2, 3]);
}
#[tokio::test]

View File

@@ -141,3 +141,41 @@ async fn test_create_flow() {
let err = procedure.on_prepare().await.unwrap_err();
assert_matches!(err, error::Error::FlowAlreadyExists { .. });
}
#[tokio::test]
async fn test_create_flow_same_source_and_sink_table() {
let table_id = 1024;
let table_name = TableName::new(DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, "same_table");
// Use the same table for both source and sink
let source_table_names = vec![table_name.clone()];
let sink_table_name = table_name.clone();
let node_manager = Arc::new(MockFlownodeManager::new(NaiveFlownodeHandler));
let ddl_context = new_ddl_context(node_manager);
// Create the table first so it exists
let task = test_create_table_task("same_table", table_id);
ddl_context
.table_metadata_manager
.create_table_metadata(
task.table_info.clone(),
TableRouteValue::physical(vec![]),
HashMap::new(),
)
.await
.unwrap();
// Try to create a flow with same source and sink table - should fail
let task = test_create_flow_task("my_flow", source_table_names, sink_table_name, false);
let query_ctx = QueryContext::arc().into();
let mut procedure = CreateFlowProcedure::new(task, query_ctx, ddl_context);
let err = procedure.on_prepare().await.unwrap_err();
assert_matches!(err, error::Error::Unsupported { .. });
// Verify the error message contains information about the same table
if let error::Error::Unsupported { operation, .. } = &err {
assert!(operation.contains("source and sink table being the same"));
assert!(operation.contains("same_table"));
}
}

View File

@@ -23,15 +23,18 @@ use common_error::ext::ErrorExt;
use common_error::status_code::StatusCode;
use common_procedure::{Context as ProcedureContext, Procedure, ProcedureId, Status};
use common_procedure_test::MockContextProvider;
use store_api::metric_engine_consts::MANIFEST_INFO_EXTENSION_KEY;
use store_api::metadata::ColumnMetadata;
use store_api::metric_engine_consts::{ALTER_PHYSICAL_EXTENSION_KEY, MANIFEST_INFO_EXTENSION_KEY};
use store_api::region_engine::RegionManifestInfo;
use store_api::storage::consts::ReservedColumnId;
use store_api::storage::RegionId;
use tokio::sync::mpsc;
use crate::ddl::create_logical_tables::CreateLogicalTablesProcedure;
use crate::ddl::test_util::datanode_handler::{DatanodeWatcher, NaiveDatanodeHandler};
use crate::ddl::test_util::{
create_physical_table_metadata, test_create_logical_table_task, test_create_physical_table_task,
assert_column_name, create_physical_table_metadata, get_raw_table_info, test_column_metadatas,
test_create_logical_table_task, test_create_physical_table_task,
};
use crate::ddl::TableMetadata;
use crate::error::{Error, Result};
@@ -39,6 +42,54 @@ use crate::key::table_route::{PhysicalTableRouteValue, TableRouteValue};
use crate::rpc::router::{Region, RegionRoute};
use crate::test_util::{new_ddl_context, MockDatanodeManager};
fn make_creates_request_handler(
column_metadatas: Vec<ColumnMetadata>,
) -> impl Fn(Peer, RegionRequest) -> Result<RegionResponse> {
move |_peer, request| {
let _ = _peer;
if let region_request::Body::Creates(_) = request.body.unwrap() {
let mut response = RegionResponse::new(0);
// Default region id for physical table.
let region_id = RegionId::new(1024, 1);
response.extensions.insert(
MANIFEST_INFO_EXTENSION_KEY.to_string(),
RegionManifestInfo::encode_list(&[(
region_id,
RegionManifestInfo::metric(1, 0, 2, 0),
)])
.unwrap(),
);
response.extensions.insert(
ALTER_PHYSICAL_EXTENSION_KEY.to_string(),
ColumnMetadata::encode_list(&column_metadatas).unwrap(),
);
return Ok(response);
}
Ok(RegionResponse::new(0))
}
}
fn assert_creates_request(
peer: Peer,
request: RegionRequest,
expected_peer_id: u64,
expected_region_ids: &[RegionId],
) {
assert_eq!(peer.id, expected_peer_id,);
let Some(region_request::Body::Creates(req)) = request.body else {
unreachable!();
};
for (i, region_id) in expected_region_ids.iter().enumerate() {
assert_eq!(
req.requests[i].region_id,
*region_id,
"actual region id: {}",
RegionId::from_u64(req.requests[i].region_id)
);
}
}
#[tokio::test]
async fn test_on_prepare_physical_table_not_found() {
let node_manager = Arc::new(MockDatanodeManager::new(()));
@@ -227,7 +278,12 @@ async fn test_on_prepare_part_logical_tables_exist() {
#[tokio::test]
async fn test_on_create_metadata() {
let node_manager = Arc::new(MockDatanodeManager::new(NaiveDatanodeHandler));
common_telemetry::init_default_ut_logging();
let (tx, mut rx) = mpsc::channel(8);
let column_metadatas = test_column_metadatas(&["host", "cpu"]);
let datanode_handler =
DatanodeWatcher::new(tx).with_handler(make_creates_request_handler(column_metadatas));
let node_manager = Arc::new(MockDatanodeManager::new(datanode_handler));
let ddl_context = new_ddl_context(node_manager);
// Prepares physical table metadata.
let mut create_physical_table_task = test_create_physical_table_task("phy_table");
@@ -255,7 +311,7 @@ async fn test_on_create_metadata() {
let mut procedure = CreateLogicalTablesProcedure::new(
vec![task, yet_another_task],
physical_table_id,
ddl_context,
ddl_context.clone(),
);
let status = procedure.on_prepare().await.unwrap();
assert_matches!(
@@ -274,11 +330,42 @@ async fn test_on_create_metadata() {
let status = procedure.execute(&ctx).await.unwrap();
let table_ids = status.downcast_output_ref::<Vec<u32>>().unwrap();
assert_eq!(*table_ids, vec![1025, 1026]);
let (peer, request) = rx.try_recv().unwrap();
rx.try_recv().unwrap_err();
assert_creates_request(
peer,
request,
0,
&[RegionId::new(1025, 0), RegionId::new(1026, 0)],
);
let table_info = get_raw_table_info(&ddl_context, table_id).await;
assert_column_name(
&table_info,
&["ts", "value", "__table_id", "__tsid", "host", "cpu"],
);
assert_eq!(
table_info.meta.column_ids,
vec![
0,
1,
ReservedColumnId::table_id(),
ReservedColumnId::tsid(),
2,
3
]
);
}
#[tokio::test]
async fn test_on_create_metadata_part_logical_tables_exist() {
let node_manager = Arc::new(MockDatanodeManager::new(NaiveDatanodeHandler));
common_telemetry::init_default_ut_logging();
let (tx, mut rx) = mpsc::channel(8);
let column_metadatas = test_column_metadatas(&["host", "cpu"]);
let datanode_handler =
DatanodeWatcher::new(tx).with_handler(make_creates_request_handler(column_metadatas));
let node_manager = Arc::new(MockDatanodeManager::new(datanode_handler));
let ddl_context = new_ddl_context(node_manager);
// Prepares physical table metadata.
let mut create_physical_table_task = test_create_physical_table_task("phy_table");
@@ -317,7 +404,7 @@ async fn test_on_create_metadata_part_logical_tables_exist() {
let mut procedure = CreateLogicalTablesProcedure::new(
vec![task, non_exist_task],
physical_table_id,
ddl_context,
ddl_context.clone(),
);
let status = procedure.on_prepare().await.unwrap();
assert_matches!(
@@ -336,6 +423,27 @@ async fn test_on_create_metadata_part_logical_tables_exist() {
let status = procedure.execute(&ctx).await.unwrap();
let table_ids = status.downcast_output_ref::<Vec<u32>>().unwrap();
assert_eq!(*table_ids, vec![8192, 1025]);
let (peer, request) = rx.try_recv().unwrap();
rx.try_recv().unwrap_err();
assert_creates_request(peer, request, 0, &[RegionId::new(1025, 0)]);
let table_info = get_raw_table_info(&ddl_context, table_id).await;
assert_column_name(
&table_info,
&["ts", "value", "__table_id", "__tsid", "host", "cpu"],
);
assert_eq!(
table_info.meta.column_ids,
vec![
0,
1,
ReservedColumnId::table_id(),
ReservedColumnId::tsid(),
2,
3
]
);
}
#[tokio::test]
@@ -399,27 +507,13 @@ async fn test_on_create_metadata_err() {
assert!(!error.is_retry_later());
}
fn creates_request_handler(_peer: Peer, request: RegionRequest) -> Result<RegionResponse> {
if let region_request::Body::Creates(_) = request.body.unwrap() {
let mut response = RegionResponse::new(0);
// Default region id for physical table.
let region_id = RegionId::new(1024, 1);
response.extensions.insert(
MANIFEST_INFO_EXTENSION_KEY.to_string(),
RegionManifestInfo::encode_list(&[(region_id, RegionManifestInfo::metric(1, 0, 2, 0))])
.unwrap(),
);
return Ok(response);
}
Ok(RegionResponse::new(0))
}
#[tokio::test]
async fn test_on_submit_create_request() {
common_telemetry::init_default_ut_logging();
let (tx, mut rx) = mpsc::channel(8);
let handler = DatanodeWatcher::new(tx).with_handler(creates_request_handler);
let column_metadatas = test_column_metadatas(&["host", "cpu"]);
let handler =
DatanodeWatcher::new(tx).with_handler(make_creates_request_handler(column_metadatas));
let node_manager = Arc::new(MockDatanodeManager::new(handler));
let ddl_context = new_ddl_context(node_manager);
let mut create_physical_table_task = test_create_physical_table_task("phy_table");

View File

@@ -16,7 +16,9 @@ use std::assert_matches::assert_matches;
use std::collections::HashMap;
use std::sync::Arc;
use api::v1::meta::Partition;
use api::region::RegionResponse;
use api::v1::meta::{Partition, Peer};
use api::v1::region::{region_request, RegionRequest};
use api::v1::{ColumnDataType, SemanticType};
use common_error::ext::ErrorExt;
use common_error::status_code::StatusCode;
@@ -24,7 +26,12 @@ use common_procedure::{Context as ProcedureContext, Procedure, ProcedureId, Stat
use common_procedure_test::{
execute_procedure_until, execute_procedure_until_done, MockContextProvider,
};
use datatypes::prelude::ConcreteDataType;
use datatypes::schema::ColumnSchema;
use store_api::metadata::ColumnMetadata;
use store_api::metric_engine_consts::TABLE_COLUMN_METADATA_EXTENSION_KEY;
use store_api::storage::RegionId;
use tokio::sync::mpsc;
use crate::ddl::create_table::{CreateTableProcedure, CreateTableState};
use crate::ddl::test_util::columns::TestColumnDefBuilder;
@@ -32,14 +39,73 @@ use crate::ddl::test_util::create_table::{
build_raw_table_info_from_expr, TestCreateTableExprBuilder,
};
use crate::ddl::test_util::datanode_handler::{
NaiveDatanodeHandler, RetryErrorDatanodeHandler, UnexpectedErrorDatanodeHandler,
DatanodeWatcher, NaiveDatanodeHandler, RetryErrorDatanodeHandler,
UnexpectedErrorDatanodeHandler,
};
use crate::error::Error;
use crate::ddl::test_util::{assert_column_name, get_raw_table_info};
use crate::error::{Error, Result};
use crate::key::table_route::TableRouteValue;
use crate::kv_backend::memory::MemoryKvBackend;
use crate::rpc::ddl::CreateTableTask;
use crate::test_util::{new_ddl_context, new_ddl_context_with_kv_backend, MockDatanodeManager};
fn create_request_handler(_peer: Peer, request: RegionRequest) -> Result<RegionResponse> {
let _ = _peer;
if let region_request::Body::Create(_) = request.body.unwrap() {
let mut response = RegionResponse::new(0);
response.extensions.insert(
TABLE_COLUMN_METADATA_EXTENSION_KEY.to_string(),
ColumnMetadata::encode_list(&[
ColumnMetadata {
column_schema: ColumnSchema::new(
"ts",
ConcreteDataType::timestamp_millisecond_datatype(),
false,
),
semantic_type: SemanticType::Timestamp,
column_id: 0,
},
ColumnMetadata {
column_schema: ColumnSchema::new(
"host",
ConcreteDataType::float64_datatype(),
false,
),
semantic_type: SemanticType::Tag,
column_id: 1,
},
ColumnMetadata {
column_schema: ColumnSchema::new(
"cpu",
ConcreteDataType::float64_datatype(),
false,
),
semantic_type: SemanticType::Tag,
column_id: 2,
},
])
.unwrap(),
);
return Ok(response);
}
Ok(RegionResponse::new(0))
}
fn assert_create_request(
peer: Peer,
request: RegionRequest,
expected_peer_id: u64,
expected_region_id: RegionId,
) {
assert_eq!(peer.id, expected_peer_id);
let Some(region_request::Body::Create(req)) = request.body else {
unreachable!();
};
assert_eq!(req.region_id, expected_region_id);
}
pub(crate) fn test_create_table_task(name: &str) -> CreateTableTask {
let create_table = TestCreateTableExprBuilder::default()
.column_defs([
@@ -230,11 +296,13 @@ async fn test_on_create_metadata_error() {
#[tokio::test]
async fn test_on_create_metadata() {
common_telemetry::init_default_ut_logging();
let node_manager = Arc::new(MockDatanodeManager::new(NaiveDatanodeHandler));
let (tx, mut rx) = mpsc::channel(8);
let datanode_handler = DatanodeWatcher::new(tx).with_handler(create_request_handler);
let node_manager = Arc::new(MockDatanodeManager::new(datanode_handler));
let ddl_context = new_ddl_context(node_manager);
let task = test_create_table_task("foo");
assert!(!task.create_table.create_if_not_exists);
let mut procedure = CreateTableProcedure::new(task, ddl_context);
let mut procedure = CreateTableProcedure::new(task, ddl_context.clone());
procedure.on_prepare().await.unwrap();
let ctx = ProcedureContext {
procedure_id: ProcedureId::random(),
@@ -243,8 +311,16 @@ async fn test_on_create_metadata() {
procedure.execute(&ctx).await.unwrap();
// Triggers procedure to create table metadata
let status = procedure.execute(&ctx).await.unwrap();
let table_id = status.downcast_output_ref::<u32>().unwrap();
assert_eq!(*table_id, 1024);
let table_id = *status.downcast_output_ref::<u32>().unwrap();
assert_eq!(table_id, 1024);
let (peer, request) = rx.try_recv().unwrap();
rx.try_recv().unwrap_err();
assert_create_request(peer, request, 0, RegionId::new(table_id, 0));
let table_info = get_raw_table_info(&ddl_context, table_id).await;
assert_column_name(&table_info, &["ts", "host", "cpu"]);
assert_eq!(table_info.meta.column_ids, vec![0, 1, 2]);
}
#[tokio::test]

View File

@@ -12,6 +12,10 @@
// See the License for the specific language governing permissions and
// limitations under the License.
pub(crate) mod raw_table_info;
pub(crate) mod table_id;
pub(crate) mod table_info;
use std::collections::HashMap;
use std::fmt::Debug;
@@ -29,6 +33,7 @@ use common_telemetry::{error, info, warn};
use common_wal::options::WalOptions;
use futures::future::join_all;
use snafu::{ensure, OptionExt, ResultExt};
use store_api::metadata::ColumnMetadata;
use store_api::metric_engine_consts::{LOGICAL_TABLE_METADATA_KEY, MANIFEST_INFO_EXTENSION_KEY};
use store_api::region_engine::RegionManifestInfo;
use store_api::storage::{RegionId, RegionNumber};
@@ -37,8 +42,8 @@ use table::table_reference::TableReference;
use crate::ddl::{DdlContext, DetectingRegion};
use crate::error::{
self, Error, OperateDatanodeSnafu, ParseWalOptionsSnafu, Result, TableNotFoundSnafu,
UnsupportedSnafu,
self, DecodeJsonSnafu, Error, MetadataCorruptionSnafu, OperateDatanodeSnafu,
ParseWalOptionsSnafu, Result, TableNotFoundSnafu, UnsupportedSnafu,
};
use crate::key::datanode_table::DatanodeTableValue;
use crate::key::table_name::TableNameKey;
@@ -314,11 +319,23 @@ pub fn parse_manifest_infos_from_extensions(
Ok(data_manifest_version)
}
/// Parses column metadatas from extensions.
pub fn parse_column_metadatas(
extensions: &HashMap<String, Vec<u8>>,
key: &str,
) -> Result<Vec<ColumnMetadata>> {
let value = extensions.get(key).context(error::UnexpectedSnafu {
err_msg: format!("column metadata extension not found: {}", key),
})?;
let column_metadatas = ColumnMetadata::decode_list(value).context(error::SerdeJsonSnafu {})?;
Ok(column_metadatas)
}
/// Sync follower regions on datanodes.
pub async fn sync_follower_regions(
context: &DdlContext,
table_id: TableId,
results: Vec<RegionResponse>,
results: &[RegionResponse],
region_routes: &[RegionRoute],
engine: &str,
) -> Result<()> {
@@ -331,7 +348,7 @@ pub async fn sync_follower_regions(
}
let results = results
.into_iter()
.iter()
.map(|response| parse_manifest_infos_from_extensions(&response.extensions))
.collect::<Result<Vec<_>>>()?
.into_iter()
@@ -418,6 +435,38 @@ pub async fn sync_follower_regions(
Ok(())
}
/// Extracts column metadatas from extensions.
pub fn extract_column_metadatas(
results: &mut [RegionResponse],
key: &str,
) -> Result<Option<Vec<ColumnMetadata>>> {
let schemas = results
.iter_mut()
.map(|r| r.extensions.remove(key))
.collect::<Vec<_>>();
if schemas.is_empty() {
return Ok(None);
}
// Verify all the physical schemas are the same
// Safety: previous check ensures this vec is not empty
let first = schemas.first().unwrap();
ensure!(
schemas.iter().all(|x| x == first),
MetadataCorruptionSnafu {
err_msg: "The table column metadata schemas from datanodes are not the same."
}
);
if let Some(first) = first {
let column_metadatas = ColumnMetadata::decode_list(first).context(DecodeJsonSnafu)?;
Ok(Some(column_metadatas))
} else {
Ok(None)
}
}
#[cfg(test)]
mod tests {
use super::*;

View File

@@ -12,9 +12,11 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::collections::HashSet;
use std::collections::{HashMap, HashSet};
use api::v1::SemanticType;
use common_telemetry::debug;
use common_telemetry::tracing::warn;
use store_api::metadata::ColumnMetadata;
use table::metadata::RawTableInfo;
@@ -23,6 +25,10 @@ pub(crate) fn build_new_physical_table_info(
mut raw_table_info: RawTableInfo,
physical_columns: &[ColumnMetadata],
) -> RawTableInfo {
debug!(
"building new physical table info for table: {}, table_id: {}",
raw_table_info.name, raw_table_info.ident.table_id
);
let existing_columns = raw_table_info
.meta
.schema
@@ -36,6 +42,8 @@ pub(crate) fn build_new_physical_table_info(
let time_index = &mut raw_table_info.meta.schema.timestamp_index;
let columns = &mut raw_table_info.meta.schema.column_schemas;
columns.clear();
let column_ids = &mut raw_table_info.meta.column_ids;
column_ids.clear();
for (idx, col) in physical_columns.iter().enumerate() {
match col.semantic_type {
@@ -50,6 +58,7 @@ pub(crate) fn build_new_physical_table_info(
}
columns.push(col.column_schema.clone());
column_ids.push(col.column_id);
}
if let Some(time_index) = *time_index {
@@ -58,3 +67,54 @@ pub(crate) fn build_new_physical_table_info(
raw_table_info
}
/// Updates the column IDs in the table info based on the provided column metadata.
///
/// This function validates that the column metadata matches the existing table schema
/// before updating the column ids. If the column metadata doesn't match the table schema,
/// the table info remains unchanged.
pub(crate) fn update_table_info_column_ids(
raw_table_info: &mut RawTableInfo,
column_metadatas: &[ColumnMetadata],
) {
let mut table_column_names = raw_table_info
.meta
.schema
.column_schemas
.iter()
.map(|c| c.name.as_str())
.collect::<Vec<_>>();
table_column_names.sort_unstable();
let mut column_names = column_metadatas
.iter()
.map(|c| c.column_schema.name.as_str())
.collect::<Vec<_>>();
column_names.sort_unstable();
if table_column_names != column_names {
warn!(
"Column metadata doesn't match the table schema for table {}, table_id: {}, column in table: {:?}, column in metadata: {:?}",
raw_table_info.name,
raw_table_info.ident.table_id,
table_column_names,
column_names,
);
return;
}
let name_to_id = column_metadatas
.iter()
.map(|c| (c.column_schema.name.clone(), c.column_id))
.collect::<HashMap<_, _>>();
let schema = &raw_table_info.meta.schema.column_schemas;
let mut column_ids = Vec::with_capacity(schema.len());
for column_schema in schema {
if let Some(id) = name_to_id.get(&column_schema.name) {
column_ids.push(*id);
}
}
raw_table_info.meta.column_ids = column_ids;
}

View File

@@ -0,0 +1,46 @@
// 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 snafu::OptionExt;
use store_api::storage::TableId;
use table::table_reference::TableReference;
use crate::error::{Result, TableNotFoundSnafu};
use crate::key::table_name::{TableNameKey, TableNameManager};
/// Get all the table ids from the table names.
///
/// Returns an error if any table does not exist.
pub(crate) async fn get_all_table_ids_by_names<'a>(
table_name_manager: &TableNameManager,
table_names: &[TableReference<'a>],
) -> Result<Vec<TableId>> {
let table_name_keys = table_names
.iter()
.map(TableNameKey::from)
.collect::<Vec<_>>();
let table_name_values = table_name_manager.batch_get(table_name_keys).await?;
let mut table_ids = Vec::with_capacity(table_name_values.len());
for (value, table_name) in table_name_values.into_iter().zip(table_names) {
let value = value
.with_context(|| TableNotFoundSnafu {
table_name: table_name.to_string(),
})?
.table_id();
table_ids.push(value);
}
Ok(table_ids)
}

View File

@@ -0,0 +1,44 @@
// 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 snafu::OptionExt;
use store_api::storage::TableId;
use table::table_reference::TableReference;
use crate::error::{Result, TableInfoNotFoundSnafu};
use crate::key::table_info::{TableInfoManager, TableInfoValue};
use crate::key::DeserializedValueWithBytes;
/// Get all table info values by table ids.
///
/// Returns an error if any table does not exist.
pub(crate) async fn get_all_table_info_values_by_table_ids<'a>(
table_info_manager: &TableInfoManager,
table_ids: &[TableId],
table_names: &[TableReference<'a>],
) -> Result<Vec<DeserializedValueWithBytes<TableInfoValue>>> {
let mut table_info_map = table_info_manager.batch_get_raw(table_ids).await?;
let mut table_info_values = Vec::with_capacity(table_ids.len());
for (table_id, table_name) in table_ids.iter().zip(table_names) {
let table_info_value =
table_info_map
.remove(table_id)
.with_context(|| TableInfoNotFoundSnafu {
table: table_name.to_string(),
})?;
table_info_values.push(table_info_value);
}
Ok(table_info_values)
}

View File

@@ -334,6 +334,7 @@ mod tests {
options: Default::default(),
region_numbers: vec![1],
partition_key_indices: vec![],
column_ids: vec![],
};
RawTableInfo {

View File

@@ -103,6 +103,26 @@ pub fn table_decoder(kv: KeyValue) -> Result<(String, TableNameValue)> {
Ok((table_name_key.table.to_string(), table_name_value))
}
impl<'a> From<&TableReference<'a>> for TableNameKey<'a> {
fn from(value: &TableReference<'a>) -> Self {
Self {
catalog: value.catalog,
schema: value.schema,
table: value.table,
}
}
}
impl<'a> From<TableReference<'a>> for TableNameKey<'a> {
fn from(value: TableReference<'a>) -> Self {
Self {
catalog: value.catalog,
schema: value.schema,
table: value.table,
}
}
}
impl<'a> From<&'a TableName> for TableNameKey<'a> {
fn from(value: &'a TableName) -> Self {
Self {

View File

@@ -14,13 +14,14 @@
use std::collections::HashMap;
use common_telemetry::debug;
use snafu::ensure;
use crate::error::{self, Result};
use crate::key::txn_helper::TxnOpGetResponseSet;
use crate::kv_backend::txn::{Compare, CompareOp, Txn, TxnOp};
use crate::kv_backend::KvBackendRef;
use crate::rpc::store::BatchGetRequest;
use crate::rpc::store::{BatchDeleteRequest, BatchGetRequest};
/// [TombstoneManager] provides the ability to:
/// - logically delete values
@@ -28,6 +29,9 @@ use crate::rpc::store::BatchGetRequest;
pub struct TombstoneManager {
kv_backend: KvBackendRef,
tombstone_prefix: String,
// Only used for testing.
#[cfg(test)]
max_txn_ops: Option<usize>,
}
const TOMBSTONE_PREFIX: &str = "__tombstone/";
@@ -35,10 +39,7 @@ const TOMBSTONE_PREFIX: &str = "__tombstone/";
impl TombstoneManager {
/// Returns [TombstoneManager].
pub fn new(kv_backend: KvBackendRef) -> Self {
Self {
kv_backend,
tombstone_prefix: TOMBSTONE_PREFIX.to_string(),
}
Self::new_with_prefix(kv_backend, TOMBSTONE_PREFIX)
}
/// Returns [TombstoneManager] with a custom tombstone prefix.
@@ -46,6 +47,8 @@ impl TombstoneManager {
Self {
kv_backend,
tombstone_prefix: prefix.to_string(),
#[cfg(test)]
max_txn_ops: None,
}
}
@@ -53,6 +56,11 @@ impl TombstoneManager {
[self.tombstone_prefix.as_bytes(), key].concat()
}
#[cfg(test)]
pub fn set_max_txn_ops(&mut self, max_txn_ops: usize) {
self.max_txn_ops = Some(max_txn_ops);
}
/// Moves value to `dest_key`.
///
/// Puts `value` to `dest_key` if the value of `src_key` equals `value`.
@@ -83,7 +91,11 @@ impl TombstoneManager {
ensure!(
keys.len() == dest_keys.len(),
error::UnexpectedSnafu {
err_msg: "The length of keys does not match the length of dest_keys."
err_msg: format!(
"The length of keys({}) does not match the length of dest_keys({}).",
keys.len(),
dest_keys.len()
),
}
);
// The key -> dest key mapping.
@@ -136,19 +148,45 @@ impl TombstoneManager {
.fail()
}
fn max_txn_ops(&self) -> usize {
#[cfg(test)]
if let Some(max_txn_ops) = self.max_txn_ops {
return max_txn_ops;
}
self.kv_backend.max_txn_ops()
}
/// Moves values to `dest_key`.
///
/// Returns the number of keys that were moved.
async fn move_values(&self, keys: Vec<Vec<u8>>, dest_keys: Vec<Vec<u8>>) -> Result<usize> {
let chunk_size = self.kv_backend.max_txn_ops() / 2;
if keys.len() > chunk_size {
let keys_chunks = keys.chunks(chunk_size).collect::<Vec<_>>();
let dest_keys_chunks = keys.chunks(chunk_size).collect::<Vec<_>>();
for (keys, dest_keys) in keys_chunks.into_iter().zip(dest_keys_chunks) {
self.move_values_inner(keys, dest_keys).await?;
ensure!(
keys.len() == dest_keys.len(),
error::UnexpectedSnafu {
err_msg: format!(
"The length of keys({}) does not match the length of dest_keys({}).",
keys.len(),
dest_keys.len()
),
}
Ok(keys.len())
);
if keys.is_empty() {
return Ok(0);
}
let chunk_size = self.max_txn_ops() / 2;
if keys.len() > chunk_size {
debug!(
"Moving values with multiple chunks, keys len: {}, chunk_size: {}",
keys.len(),
chunk_size
);
let mut moved_keys = 0;
let keys_chunks = keys.chunks(chunk_size).collect::<Vec<_>>();
let dest_keys_chunks = dest_keys.chunks(chunk_size).collect::<Vec<_>>();
for (keys, dest_keys) in keys_chunks.into_iter().zip(dest_keys_chunks) {
moved_keys += self.move_values_inner(keys, dest_keys).await?;
}
Ok(moved_keys)
} else {
self.move_values_inner(&keys, &dest_keys).await
}
@@ -196,15 +234,18 @@ impl TombstoneManager {
///
/// Returns the number of keys that were deleted.
pub async fn delete(&self, keys: Vec<Vec<u8>>) -> Result<usize> {
let operations = keys
let keys = keys
.iter()
.map(|key| TxnOp::Delete(self.to_tombstone(key)))
.map(|key| self.to_tombstone(key))
.collect::<Vec<_>>();
let txn = Txn::new().and_then(operations);
// Always success.
let _ = self.kv_backend.txn(txn).await?;
Ok(keys.len())
let num_keys = keys.len();
let _ = self
.kv_backend
.batch_delete(BatchDeleteRequest::new().with_keys(keys))
.await?;
Ok(num_keys)
}
}
@@ -392,16 +433,73 @@ mod tests {
.into_iter()
.map(|kv| (kv.key, kv.dest_key))
.unzip();
tombstone_manager
let moved_keys = tombstone_manager
.move_values(keys.clone(), dest_keys.clone())
.await
.unwrap();
assert_eq!(kvs.len(), moved_keys);
check_moved_values(kv_backend.clone(), &move_values).await;
// Moves again
tombstone_manager
let moved_keys = tombstone_manager
.move_values(keys.clone(), dest_keys.clone())
.await
.unwrap();
assert_eq!(0, moved_keys);
check_moved_values(kv_backend.clone(), &move_values).await;
}
#[tokio::test]
async fn test_move_values_with_max_txn_ops() {
common_telemetry::init_default_ut_logging();
let kv_backend = Arc::new(MemoryKvBackend::default());
let mut tombstone_manager = TombstoneManager::new(kv_backend.clone());
tombstone_manager.set_max_txn_ops(4);
let kvs = HashMap::from([
(b"bar".to_vec(), b"baz".to_vec()),
(b"foo".to_vec(), b"hi".to_vec()),
(b"baz".to_vec(), b"hello".to_vec()),
(b"qux".to_vec(), b"world".to_vec()),
(b"quux".to_vec(), b"world".to_vec()),
(b"quuux".to_vec(), b"world".to_vec()),
(b"quuuux".to_vec(), b"world".to_vec()),
(b"quuuuux".to_vec(), b"world".to_vec()),
(b"quuuuuux".to_vec(), b"world".to_vec()),
]);
for (key, value) in &kvs {
kv_backend
.put(
PutRequest::new()
.with_key(key.clone())
.with_value(value.clone()),
)
.await
.unwrap();
}
let move_values = kvs
.iter()
.map(|(key, value)| MoveValue {
key: key.clone(),
dest_key: tombstone_manager.to_tombstone(key),
value: value.clone(),
})
.collect::<Vec<_>>();
let (keys, dest_keys): (Vec<_>, Vec<_>) = move_values
.clone()
.into_iter()
.map(|kv| (kv.key, kv.dest_key))
.unzip();
let moved_keys = tombstone_manager
.move_values(keys.clone(), dest_keys.clone())
.await
.unwrap();
assert_eq!(kvs.len(), moved_keys);
check_moved_values(kv_backend.clone(), &move_values).await;
// Moves again
let moved_keys = tombstone_manager
.move_values(keys.clone(), dest_keys.clone())
.await
.unwrap();
assert_eq!(0, moved_keys);
check_moved_values(kv_backend.clone(), &move_values).await;
}
@@ -439,17 +537,19 @@ mod tests {
.unzip();
keys.push(b"non-exists".to_vec());
dest_keys.push(b"hi/non-exists".to_vec());
tombstone_manager
let moved_keys = tombstone_manager
.move_values(keys.clone(), dest_keys.clone())
.await
.unwrap();
check_moved_values(kv_backend.clone(), &move_values).await;
assert_eq!(3, moved_keys);
// Moves again
tombstone_manager
let moved_keys = tombstone_manager
.move_values(keys.clone(), dest_keys.clone())
.await
.unwrap();
check_moved_values(kv_backend.clone(), &move_values).await;
assert_eq!(0, moved_keys);
}
#[tokio::test]
@@ -490,10 +590,11 @@ mod tests {
.into_iter()
.map(|kv| (kv.key, kv.dest_key))
.unzip();
tombstone_manager
let moved_keys = tombstone_manager
.move_values(keys, dest_keys)
.await
.unwrap();
assert_eq!(kvs.len(), moved_keys);
}
#[tokio::test]
@@ -571,4 +672,24 @@ mod tests {
.unwrap();
check_moved_values(kv_backend.clone(), &move_values).await;
}
#[tokio::test]
async fn test_move_values_with_different_lengths() {
let kv_backend = Arc::new(MemoryKvBackend::default());
let tombstone_manager = TombstoneManager::new(kv_backend.clone());
let keys = vec![b"bar".to_vec(), b"foo".to_vec()];
let dest_keys = vec![b"bar".to_vec(), b"foo".to_vec(), b"baz".to_vec()];
let err = tombstone_manager
.move_values(keys, dest_keys)
.await
.unwrap_err();
assert!(err
.to_string()
.contains("The length of keys(2) does not match the length of dest_keys(3)."),);
let moved_keys = tombstone_manager.move_values(vec![], vec![]).await.unwrap();
assert_eq!(0, moved_keys);
}
}

View File

@@ -1374,6 +1374,7 @@ mod tests {
options: Default::default(),
created_on: Default::default(),
partition_key_indices: Default::default(),
column_ids: Default::default(),
};
// construct RawTableInfo

View File

@@ -178,8 +178,6 @@ pub enum Error {
StreamTimeout {
#[snafu(implicit)]
location: Location,
#[snafu(source)]
error: tokio::time::error::Elapsed,
},
#[snafu(display("RecordBatch slice index overflow: {visit_index} > {size}"))]

22
src/common/sql/Cargo.toml Normal file
View File

@@ -0,0 +1,22 @@
[package]
name = "common-sql"
version.workspace = true
edition.workspace = true
license.workspace = true
[dependencies]
common-base.workspace = true
common-datasource.workspace = true
common-decimal.workspace = true
common-error.workspace = true
common-macro.workspace = true
common-time.workspace = true
datafusion-sql.workspace = true
datatypes.workspace = true
hex = "0.4"
jsonb.workspace = true
snafu.workspace = true
sqlparser.workspace = true
[lints]
workspace = true

File diff suppressed because it is too large Load Diff

View File

@@ -0,0 +1,182 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use common_time::timezone::Timezone;
use datatypes::prelude::ConcreteDataType;
use datatypes::schema::constraint::{CURRENT_TIMESTAMP, CURRENT_TIMESTAMP_FN};
use datatypes::schema::ColumnDefaultConstraint;
pub use sqlparser::ast::{
visit_expressions_mut, visit_statements_mut, BinaryOperator, ColumnDef, ColumnOption,
ColumnOptionDef, DataType, Expr, Function, FunctionArg, FunctionArgExpr, FunctionArguments,
Ident, ObjectName, SqlOption, TableConstraint, TimezoneInfo, UnaryOperator, Value as SqlValue,
Visit, VisitMut, Visitor, VisitorMut,
};
use crate::convert::{sql_number_to_value, sql_value_to_value};
use crate::error::{Result, UnsupportedDefaultValueSnafu};
pub fn parse_column_default_constraint(
column_name: &str,
data_type: &ConcreteDataType,
opts: &[ColumnOptionDef],
timezone: Option<&Timezone>,
) -> Result<Option<ColumnDefaultConstraint>> {
if let Some(opt) = opts
.iter()
.find(|o| matches!(o.option, ColumnOption::Default(_)))
{
let default_constraint = match &opt.option {
ColumnOption::Default(Expr::Value(v)) => ColumnDefaultConstraint::Value(
sql_value_to_value(column_name, data_type, v, timezone, None, false)?,
),
ColumnOption::Default(Expr::Function(func)) => {
let mut func = format!("{func}").to_lowercase();
// normalize CURRENT_TIMESTAMP to CURRENT_TIMESTAMP()
if func == CURRENT_TIMESTAMP {
func = CURRENT_TIMESTAMP_FN.to_string();
}
// Always use lowercase for function expression
ColumnDefaultConstraint::Function(func.to_lowercase())
}
ColumnOption::Default(Expr::UnaryOp { op, expr }) => {
// Specialized process for handling numerical inputs to prevent
// overflow errors during the parsing of negative numbers,
// See https://github.com/GreptimeTeam/greptimedb/issues/4351
if let (UnaryOperator::Minus, Expr::Value(SqlValue::Number(n, _))) =
(op, expr.as_ref())
{
return Ok(Some(ColumnDefaultConstraint::Value(sql_number_to_value(
data_type,
&format!("-{n}"),
)?)));
}
if let Expr::Value(v) = &**expr {
let value =
sql_value_to_value(column_name, data_type, v, timezone, Some(*op), false)?;
ColumnDefaultConstraint::Value(value)
} else {
return UnsupportedDefaultValueSnafu {
column_name,
expr: *expr.clone(),
}
.fail();
}
}
ColumnOption::Default(others) => {
return UnsupportedDefaultValueSnafu {
column_name,
expr: others.clone(),
}
.fail();
}
_ => {
return UnsupportedDefaultValueSnafu {
column_name,
expr: Expr::Value(SqlValue::Null),
}
.fail();
}
};
Ok(Some(default_constraint))
} else {
Ok(None)
}
}
#[cfg(test)]
mod test {
use std::assert_matches::assert_matches;
use datatypes::prelude::{ConcreteDataType, Value};
use datatypes::types::BooleanType;
use super::*;
#[test]
pub fn test_parse_column_default_constraint() {
let bool_value = sqlparser::ast::Value::Boolean(true);
let opts = vec![
ColumnOptionDef {
name: None,
option: ColumnOption::Default(Expr::Value(bool_value)),
},
ColumnOptionDef {
name: None,
option: ColumnOption::NotNull,
},
];
let constraint = parse_column_default_constraint(
"coll",
&ConcreteDataType::Boolean(BooleanType),
&opts,
None,
)
.unwrap();
assert_matches!(
constraint,
Some(ColumnDefaultConstraint::Value(Value::Boolean(true)))
);
// Test negative number
let opts = vec![ColumnOptionDef {
name: None,
option: ColumnOption::Default(Expr::UnaryOp {
op: UnaryOperator::Minus,
expr: Box::new(Expr::Value(SqlValue::Number("32768".to_string(), false))),
}),
}];
let constraint = parse_column_default_constraint(
"coll",
&ConcreteDataType::int16_datatype(),
&opts,
None,
)
.unwrap();
assert_matches!(
constraint,
Some(ColumnDefaultConstraint::Value(Value::Int16(-32768)))
);
}
#[test]
fn test_incorrect_default_value_issue_3479() {
let opts = vec![ColumnOptionDef {
name: None,
option: ColumnOption::Default(Expr::Value(SqlValue::Number(
"0.047318541668048164".into(),
false,
))),
}];
let constraint = parse_column_default_constraint(
"coll",
&ConcreteDataType::float64_datatype(),
&opts,
None,
)
.unwrap()
.unwrap();
assert_eq!("0.047318541668048164", constraint.to_string());
let encoded: Vec<u8> = constraint.clone().try_into().unwrap();
let decoded = ColumnDefaultConstraint::try_from(encoded.as_ref()).unwrap();
assert_eq!(decoded, constraint);
}
}

158
src/common/sql/src/error.rs Normal file
View File

@@ -0,0 +1,158 @@
// 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::any::Any;
use common_error::ext::ErrorExt;
use common_error::status_code::StatusCode;
use common_macro::stack_trace_debug;
use common_time::timestamp::TimeUnit;
use common_time::Timestamp;
use datafusion_sql::sqlparser::ast::UnaryOperator;
use datatypes::prelude::{ConcreteDataType, Value};
use snafu::{Location, Snafu};
pub use sqlparser::ast::{Expr, Value as SqlValue};
pub type Result<T> = std::result::Result<T, Error>;
/// SQL parser errors.
// Now the error in parser does not contain backtrace to avoid generating backtrace
// every time the parser parses an invalid SQL.
#[derive(Snafu)]
#[snafu(visibility(pub))]
#[stack_trace_debug]
pub enum Error {
#[snafu(display(
"Column {} expect type: {:?}, actual: {:?}",
column_name,
expect,
actual,
))]
ColumnTypeMismatch {
column_name: String,
expect: ConcreteDataType,
actual: ConcreteDataType,
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Failed to parse value: {}", msg))]
ParseSqlValue {
msg: String,
#[snafu(implicit)]
location: Location,
},
#[snafu(display(
"Unsupported expr in default constraint: {:?} for column: {}",
expr,
column_name
))]
UnsupportedDefaultValue {
column_name: String,
expr: Expr,
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Unable to convert sql value {} to datatype {:?}", value, datatype))]
ConvertSqlValue {
value: SqlValue,
datatype: ConcreteDataType,
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Invalid sql value: {}", value))]
InvalidSqlValue {
value: String,
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Unsupported unary operator {}", unary_op))]
UnsupportedUnaryOp {
unary_op: UnaryOperator,
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Invalid unary operator {} for value {}", unary_op, value))]
InvalidUnaryOp {
unary_op: UnaryOperator,
value: Value,
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Failed to cast SQL value {} to datatype {}", sql_value, datatype))]
InvalidCast {
sql_value: sqlparser::ast::Value,
datatype: ConcreteDataType,
#[snafu(implicit)]
location: Location,
source: datatypes::error::Error,
},
#[snafu(display("Unable to convert {} to datatype {:?}", value, datatype))]
ConvertStr {
value: String,
datatype: ConcreteDataType,
#[snafu(implicit)]
location: Location,
},
#[snafu(display(
"Converting timestamp {:?} to unit {:?} overflow",
timestamp,
target_unit
))]
TimestampOverflow {
timestamp: Timestamp,
target_unit: TimeUnit,
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Datatype error: {}", source))]
Datatype {
source: datatypes::error::Error,
#[snafu(implicit)]
location: Location,
},
}
impl ErrorExt for Error {
fn status_code(&self) -> StatusCode {
use Error::*;
match self {
UnsupportedDefaultValue { .. } => StatusCode::Unsupported,
ParseSqlValue { .. } => StatusCode::InvalidSyntax,
ColumnTypeMismatch { .. }
| InvalidSqlValue { .. }
| UnsupportedUnaryOp { .. }
| InvalidUnaryOp { .. }
| InvalidCast { .. }
| ConvertStr { .. }
| TimestampOverflow { .. } => StatusCode::InvalidArguments,
Datatype { source, .. } => source.status_code(),
ConvertSqlValue { .. } => StatusCode::Unsupported,
}
}
fn as_any(&self) -> &dyn Any {
self
}
}

19
src/common/sql/src/lib.rs Normal file
View File

@@ -0,0 +1,19 @@
// 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.
#![feature(assert_matches)]
pub mod convert;
pub mod default_constraint;
pub mod error;

View File

@@ -14,6 +14,7 @@ workspace = true
[dependencies]
backtrace = "0.3"
common-error.workspace = true
common-version.workspace = true
console-subscriber = { version = "0.1", optional = true }
greptime-proto.workspace = true
humantime-serde.workspace = true

View File

@@ -403,7 +403,7 @@ pub fn init_global_logging(
resource::SERVICE_INSTANCE_ID,
node_id.unwrap_or("none".to_string()),
),
KeyValue::new(resource::SERVICE_VERSION, env!("CARGO_PKG_VERSION")),
KeyValue::new(resource::SERVICE_VERSION, common_version::version()),
KeyValue::new(resource::PROCESS_PID, std::process::id().to_string()),
]));

View File

@@ -13,6 +13,7 @@
// limitations under the License.
use std::fmt::{Debug, Display, Formatter};
use std::ops::{Bound, RangeBounds};
use serde::{Deserialize, Serialize};
@@ -303,6 +304,41 @@ impl TimestampRange {
}
}
/// Create [TimestampRange] from a timestamp tuple.
/// The tuple's two elements form the "start" and "end" (both inclusive) of the timestamp range.
impl From<(Timestamp, Timestamp)> for TimestampRange {
fn from((start, end): (Timestamp, Timestamp)) -> Self {
if start > end {
Self::empty()
} else {
Self::new_inclusive(Some(start), Some(end))
}
}
}
/// Create [TimestampRange] from Rust's "range".
impl<R: RangeBounds<Timestamp>> From<R> for TimestampRange {
fn from(r: R) -> Self {
let start = match r.start_bound() {
Bound::Included(x) => Some(*x),
Bound::Excluded(x) => x
.value()
.checked_sub(1)
.map(|v| Timestamp::new(v, x.unit())),
Bound::Unbounded => None,
};
let end = match r.end_bound() {
Bound::Included(x) => x
.value()
.checked_add(1)
.map(|v| Timestamp::new(v, x.unit())),
Bound::Excluded(x) => Some(*x),
Bound::Unbounded => None,
};
Self::from_optional(start, end)
}
}
/// Time range in milliseconds.
pub type RangeMillis = GenericRange<TimestampMillis>;
@@ -545,4 +581,75 @@ mod tests {
TimeUnit::Nanosecond
);
}
#[test]
fn test_from_timestamp_tuple() {
let timestamp_range: TimestampRange =
(Timestamp::new_millisecond(1), Timestamp::new_millisecond(3)).into();
assert_eq!(
timestamp_range,
TimestampRange::from_optional(
Some(Timestamp::new_millisecond(1)),
Some(Timestamp::new_millisecond(4))
)
);
let timestamp_range: TimestampRange =
(Timestamp::new_millisecond(1), Timestamp::new_millisecond(1)).into();
assert_eq!(
timestamp_range,
TimestampRange::from_optional(
Some(Timestamp::new_millisecond(1)),
Some(Timestamp::new_millisecond(2))
)
);
let timestamp_range: TimestampRange =
(Timestamp::new_second(1), Timestamp::new_millisecond(3)).into();
assert_eq!(timestamp_range, TimestampRange::empty());
}
#[test]
fn test_from_timestamp_range() {
let timestamp_range: TimestampRange =
(Timestamp::new_millisecond(1)..Timestamp::new_millisecond(3)).into();
assert_eq!(
timestamp_range,
TimestampRange::from_optional(
Some(Timestamp::new_millisecond(1)),
Some(Timestamp::new_millisecond(3))
)
);
let timestamp_range: TimestampRange =
(Timestamp::new_millisecond(1)..=Timestamp::new_millisecond(3)).into();
assert_eq!(
timestamp_range,
TimestampRange::from_optional(
Some(Timestamp::new_millisecond(1)),
Some(Timestamp::new_millisecond(4))
)
);
let timestamp_range: TimestampRange = (Timestamp::new_millisecond(1)..).into();
assert_eq!(
timestamp_range,
TimestampRange::from_optional(Some(Timestamp::new_millisecond(1)), None)
);
let timestamp_range: TimestampRange = (..Timestamp::new_millisecond(3)).into();
assert_eq!(
timestamp_range,
TimestampRange::from_optional(None, Some(Timestamp::new_millisecond(3)))
);
let timestamp_range: TimestampRange = (..=Timestamp::new_millisecond(3)).into();
assert_eq!(
timestamp_range,
TimestampRange::from_optional(None, Some(Timestamp::new_millisecond(4)))
);
let timestamp_range: TimestampRange = (..).into();
assert_eq!(timestamp_range, TimestampRange::min_to_max(),);
}
}

View File

@@ -17,4 +17,5 @@ shadow-rs.workspace = true
[build-dependencies]
build-data = "0.2"
cargo-manifest = "0.19"
shadow-rs.workspace = true

View File

@@ -14,8 +14,10 @@
use std::collections::BTreeSet;
use std::env;
use std::path::PathBuf;
use build_data::{format_timestamp, get_source_time};
use cargo_manifest::Manifest;
use shadow_rs::{BuildPattern, ShadowBuilder, CARGO_METADATA, CARGO_TREE};
fn main() -> shadow_rs::SdResult<()> {
@@ -33,6 +35,24 @@ fn main() -> shadow_rs::SdResult<()> {
// solve the problem where the "CARGO_MANIFEST_DIR" is not what we want when this repo is
// made as a submodule in another repo.
let src_path = env::var("CARGO_WORKSPACE_DIR").or_else(|_| env::var("CARGO_MANIFEST_DIR"))?;
let manifest = Manifest::from_path(PathBuf::from(&src_path).join("Cargo.toml"))
.expect("Failed to parse Cargo.toml");
if let Some(product_version) = manifest.workspace.as_ref().and_then(|w| {
w.metadata.as_ref().and_then(|m| {
m.get("greptime")
.and_then(|g| g.get("product_version").and_then(|v| v.as_str()))
})
}) {
println!(
"cargo:rustc-env=GREPTIME_PRODUCT_VERSION={}",
product_version
);
} else {
let version = env::var("CARGO_PKG_VERSION").unwrap();
println!("cargo:rustc-env=GREPTIME_PRODUCT_VERSION={}", version,);
}
let out_path = env::var("OUT_DIR")?;
let _ = ShadowBuilder::builder()

View File

@@ -105,13 +105,17 @@ pub const fn build_info() -> BuildInfo {
build_time: env!("BUILD_TIMESTAMP"),
rustc: build::RUST_VERSION,
target: build::BUILD_TARGET,
version: build::PKG_VERSION,
version: env!("GREPTIME_PRODUCT_VERSION"),
}
}
const BUILD_INFO: BuildInfo = build_info();
pub const fn version() -> &'static str {
BUILD_INFO.version
}
pub const fn verbose_version() -> &'static str {
const_format::formatcp!(
"\nbranch: {}\ncommit: {}\nclean: {}\nversion: {}",
BUILD_INFO.branch,

View File

@@ -86,26 +86,33 @@ impl RegionAliveKeeper {
/// Add the countdown task for a specific region.
/// It will be ignored if the task exists.
pub async fn register_region(&self, region_id: RegionId) {
if self.find_handle(region_id).await.is_some() {
return;
}
let handle = Arc::new(CountdownTaskHandle::new(
self.region_server.clone(),
self.countdown_task_handler_ext.clone(),
region_id,
));
let mut handles = self.tasks.lock().await;
let _ = handles.insert(region_id, handle.clone());
let should_start = {
let mut handles = self.tasks.lock().await;
if self.started.load(Ordering::Relaxed) {
// Check if already exists, return early if so
if handles.contains_key(&region_id) {
return;
}
// Insert new handle
handles.insert(region_id, handle.clone());
// Return whether we should start (check state inside lock)
self.started.load(Ordering::Relaxed)
};
if should_start {
handle.start(self.heartbeat_interval_millis).await;
info!("Region alive countdown for region {region_id} is started!",);
info!("Region alive countdown for region {region_id} is started!");
} else {
info!(
"Region alive countdown for region {region_id} is registered but not started yet!",
"Region alive countdown for region {region_id} is registered but not started yet!"
);
}
}

View File

@@ -40,7 +40,7 @@ use log_store::raft_engine::log_store::RaftEngineLogStore;
use meta_client::MetaClientRef;
use metric_engine::engine::MetricEngine;
use mito2::config::MitoConfig;
use mito2::engine::MitoEngine;
use mito2::engine::{MitoEngine, MitoEngineBuilder};
use object_store::manager::{ObjectStoreManager, ObjectStoreManagerRef};
use object_store::util::normalize_dir;
use query::dummy_catalog::TableProviderFactoryRef;
@@ -162,6 +162,8 @@ pub struct DatanodeBuilder {
meta_client: Option<MetaClientRef>,
kv_backend: KvBackendRef,
cache_registry: Option<Arc<LayeredCacheRegistry>>,
#[cfg(feature = "enterprise")]
extension_range_provider_factory: Option<mito2::extension::BoxedExtensionRangeProviderFactory>,
}
impl DatanodeBuilder {
@@ -173,6 +175,8 @@ impl DatanodeBuilder {
meta_client: None,
kv_backend,
cache_registry: None,
#[cfg(feature = "enterprise")]
extension_range_provider_factory: None,
}
}
@@ -199,6 +203,15 @@ impl DatanodeBuilder {
self
}
#[cfg(feature = "enterprise")]
pub fn with_extension_range_provider(
&mut self,
extension_range_provider_factory: mito2::extension::BoxedExtensionRangeProviderFactory,
) -> &mut Self {
self.extension_range_provider_factory = Some(extension_range_provider_factory);
self
}
pub async fn build(mut self) -> Result<Datanode> {
let node_id = self.opts.node_id.context(MissingNodeIdSnafu)?;
@@ -340,7 +353,7 @@ impl DatanodeBuilder {
}
async fn new_region_server(
&self,
&mut self,
schema_metadata_manager: SchemaMetadataManagerRef,
event_listener: RegionServerEventListenerRef,
) -> Result<RegionServer> {
@@ -376,13 +389,13 @@ impl DatanodeBuilder {
);
let object_store_manager = Self::build_object_store_manager(&opts.storage).await?;
let engines = Self::build_store_engines(
opts,
object_store_manager,
schema_metadata_manager,
self.plugins.clone(),
)
.await?;
let engines = self
.build_store_engines(
object_store_manager,
schema_metadata_manager,
self.plugins.clone(),
)
.await?;
for engine in engines {
region_server.register_engine(engine);
}
@@ -394,7 +407,7 @@ impl DatanodeBuilder {
/// Builds [RegionEngineRef] from `store_engine` section in `opts`
async fn build_store_engines(
opts: &DatanodeOptions,
&mut self,
object_store_manager: ObjectStoreManagerRef,
schema_metadata_manager: SchemaMetadataManagerRef,
plugins: Plugins,
@@ -403,7 +416,7 @@ impl DatanodeBuilder {
let mut mito_engine_config = MitoConfig::default();
let mut file_engine_config = file_engine::config::EngineConfig::default();
for engine in &opts.region_engine {
for engine in &self.opts.region_engine {
match engine {
RegionEngineConfig::Mito(config) => {
mito_engine_config = config.clone();
@@ -417,14 +430,14 @@ impl DatanodeBuilder {
}
}
let mito_engine = Self::build_mito_engine(
opts,
object_store_manager.clone(),
mito_engine_config,
schema_metadata_manager.clone(),
plugins.clone(),
)
.await?;
let mito_engine = self
.build_mito_engine(
object_store_manager.clone(),
mito_engine_config,
schema_metadata_manager.clone(),
plugins.clone(),
)
.await?;
let metric_engine = MetricEngine::try_new(mito_engine.clone(), metric_engine_config)
.context(BuildMetricEngineSnafu)?;
@@ -443,12 +456,13 @@ impl DatanodeBuilder {
/// Builds [MitoEngine] according to options.
async fn build_mito_engine(
opts: &DatanodeOptions,
&mut self,
object_store_manager: ObjectStoreManagerRef,
mut config: MitoConfig,
schema_metadata_manager: SchemaMetadataManagerRef,
plugins: Plugins,
) -> Result<MitoEngine> {
let opts = &self.opts;
if opts.storage.is_object_storage() {
// Enable the write cache when setting object storage
config.enable_write_cache = true;
@@ -456,17 +470,27 @@ impl DatanodeBuilder {
}
let mito_engine = match &opts.wal {
DatanodeWalConfig::RaftEngine(raft_engine_config) => MitoEngine::new(
&opts.storage.data_home,
config,
Self::build_raft_engine_log_store(&opts.storage.data_home, raft_engine_config)
.await?,
object_store_manager,
schema_metadata_manager,
plugins,
)
.await
.context(BuildMitoEngineSnafu)?,
DatanodeWalConfig::RaftEngine(raft_engine_config) => {
let log_store =
Self::build_raft_engine_log_store(&opts.storage.data_home, raft_engine_config)
.await?;
let builder = MitoEngineBuilder::new(
&opts.storage.data_home,
config,
log_store,
object_store_manager,
schema_metadata_manager,
plugins,
);
#[cfg(feature = "enterprise")]
let builder = builder.with_extension_range_provider_factory(
self.extension_range_provider_factory.take(),
);
builder.try_build().await.context(BuildMitoEngineSnafu)?
}
DatanodeWalConfig::Kafka(kafka_config) => {
if kafka_config.create_index && opts.node_id.is_none() {
warn!("The WAL index creation only available in distributed mode.")
@@ -488,16 +512,21 @@ impl DatanodeBuilder {
None
};
MitoEngine::new(
let builder = MitoEngineBuilder::new(
&opts.storage.data_home,
config,
Self::build_kafka_log_store(kafka_config, global_index_collector).await?,
object_store_manager,
schema_metadata_manager,
plugins,
)
.await
.context(BuildMitoEngineSnafu)?
);
#[cfg(feature = "enterprise")]
let builder = builder.with_extension_range_provider_factory(
self.extension_range_provider_factory.take(),
);
builder.try_build().await.context(BuildMitoEngineSnafu)?
}
};
Ok(mito_engine)

View File

@@ -402,6 +402,9 @@ pub enum Error {
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Not yet implemented: {what}"))]
NotYetImplemented { what: String },
}
pub type Result<T> = std::result::Result<T, Error>;
@@ -456,7 +459,7 @@ impl ErrorExt for Error {
OpenLogStore { source, .. } => source.status_code(),
MetaClientInit { source, .. } => source.status_code(),
UnsupportedOutput { .. } => StatusCode::Unsupported,
UnsupportedOutput { .. } | NotYetImplemented { .. } => StatusCode::Unsupported,
HandleRegionRequest { source, .. }
| GetRegionMetadata { source, .. }
| HandleBatchOpenRequest { source, .. }

View File

@@ -40,7 +40,7 @@ use store_api::storage::{RegionId, ScanRequest, SequenceNumber};
use table::TableRef;
use tokio::sync::mpsc::{Receiver, Sender};
use crate::error::Error;
use crate::error::{Error, NotYetImplementedSnafu};
use crate::event_listener::NoopRegionServerEventListener;
use crate::region_server::RegionServer;
@@ -232,7 +232,9 @@ impl RegionEngine for MockRegionEngine {
_region_id: RegionId,
_request: ScanRequest,
) -> Result<RegionScannerRef, BoxedError> {
unimplemented!()
Err(BoxedError::new(
NotYetImplementedSnafu { what: "blah" }.build(),
))
}
async fn get_metadata(&self, region_id: RegionId) -> Result<RegionMetadataRef, BoxedError> {

View File

@@ -31,9 +31,10 @@ pub use crate::schema::column_schema::{
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,
COLUMN_FULLTEXT_OPT_KEY_CASE_SENSITIVE, COLUMN_FULLTEXT_OPT_KEY_FALSE_POSITIVE_RATE,
COLUMN_FULLTEXT_OPT_KEY_GRANULARITY, COLUMN_SKIPPING_INDEX_OPT_KEY_FALSE_POSITIVE_RATE,
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,
};
pub use crate::schema::constraint::ColumnDefaultConstraint;
pub use crate::schema::raw::RawSchema;

View File

@@ -47,13 +47,18 @@ 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";
pub const COLUMN_FULLTEXT_OPT_KEY_GRANULARITY: &str = "granularity";
pub const COLUMN_FULLTEXT_OPT_KEY_FALSE_POSITIVE_RATE: &str = "false_positive_rate";
/// Keys used in SKIPPING index options
pub const COLUMN_SKIPPING_INDEX_OPT_KEY_GRANULARITY: &str = "granularity";
pub const COLUMN_SKIPPING_INDEX_OPT_KEY_FALSE_POSITIVE_RATE: &str = "false_positive_rate";
pub const COLUMN_SKIPPING_INDEX_OPT_KEY_TYPE: &str = "type";
pub const DEFAULT_GRANULARITY: u32 = 10240;
pub const DEFAULT_FALSE_POSITIVE_RATE: f64 = 0.01;
/// Schema of a column, used as an immutable struct.
#[derive(Clone, PartialEq, Eq, Serialize, Deserialize)]
pub struct ColumnSchema {
@@ -504,7 +509,7 @@ impl TryFrom<&ColumnSchema> for Field {
}
/// Fulltext options for a column.
#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize, Default, Visit, VisitMut)]
#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize, Visit, VisitMut)]
#[serde(rename_all = "kebab-case")]
pub struct FulltextOptions {
/// Whether the fulltext index is enabled.
@@ -518,6 +523,92 @@ pub struct FulltextOptions {
/// The fulltext backend to use.
#[serde(default)]
pub backend: FulltextBackend,
/// The granularity of the fulltext index (for bloom backend only)
#[serde(default = "fulltext_options_default_granularity")]
pub granularity: u32,
/// The false positive rate of the fulltext index (for bloom backend only)
#[serde(default = "index_options_default_false_positive_rate_in_10000")]
pub false_positive_rate_in_10000: u32,
}
fn fulltext_options_default_granularity() -> u32 {
DEFAULT_GRANULARITY
}
fn index_options_default_false_positive_rate_in_10000() -> u32 {
(DEFAULT_FALSE_POSITIVE_RATE * 10000.0) as u32
}
impl FulltextOptions {
/// Creates a new fulltext options.
pub fn new(
enable: bool,
analyzer: FulltextAnalyzer,
case_sensitive: bool,
backend: FulltextBackend,
granularity: u32,
false_positive_rate: f64,
) -> Result<Self> {
ensure!(
0.0 < false_positive_rate && false_positive_rate <= 1.0,
error::InvalidFulltextOptionSnafu {
msg: format!(
"Invalid false positive rate: {false_positive_rate}, expected: 0.0 < rate <= 1.0"
),
}
);
ensure!(
granularity > 0,
error::InvalidFulltextOptionSnafu {
msg: format!("Invalid granularity: {granularity}, expected: positive integer"),
}
);
Ok(Self::new_unchecked(
enable,
analyzer,
case_sensitive,
backend,
granularity,
false_positive_rate,
))
}
/// Creates a new fulltext options without checking `false_positive_rate` and `granularity`.
pub fn new_unchecked(
enable: bool,
analyzer: FulltextAnalyzer,
case_sensitive: bool,
backend: FulltextBackend,
granularity: u32,
false_positive_rate: f64,
) -> Self {
Self {
enable,
analyzer,
case_sensitive,
backend,
granularity,
false_positive_rate_in_10000: (false_positive_rate * 10000.0) as u32,
}
}
/// Gets the false positive rate.
pub fn false_positive_rate(&self) -> f64 {
self.false_positive_rate_in_10000 as f64 / 10000.0
}
}
impl Default for FulltextOptions {
fn default() -> Self {
Self::new_unchecked(
false,
FulltextAnalyzer::default(),
false,
FulltextBackend::default(),
DEFAULT_GRANULARITY,
DEFAULT_FALSE_POSITIVE_RATE,
)
}
}
impl fmt::Display for FulltextOptions {
@@ -527,6 +618,10 @@ impl fmt::Display for FulltextOptions {
write!(f, ", analyzer={}", self.analyzer)?;
write!(f, ", case_sensitive={}", self.case_sensitive)?;
write!(f, ", backend={}", self.backend)?;
if self.backend == FulltextBackend::Bloom {
write!(f, ", granularity={}", self.granularity)?;
write!(f, ", false_positive_rate={}", self.false_positive_rate())?;
}
}
Ok(())
}
@@ -611,6 +706,45 @@ impl TryFrom<HashMap<String, String>> for FulltextOptions {
}
}
if fulltext_options.backend == FulltextBackend::Bloom {
// Parse granularity with default value 10240
let granularity = match options.get(COLUMN_FULLTEXT_OPT_KEY_GRANULARITY) {
Some(value) => value
.parse::<u32>()
.ok()
.filter(|&v| v > 0)
.ok_or_else(|| {
error::InvalidFulltextOptionSnafu {
msg: format!(
"Invalid granularity: {value}, expected: positive integer"
),
}
.build()
})?,
None => DEFAULT_GRANULARITY,
};
fulltext_options.granularity = granularity;
// Parse false positive rate with default value 0.01
let false_positive_rate = match options.get(COLUMN_FULLTEXT_OPT_KEY_FALSE_POSITIVE_RATE)
{
Some(value) => value
.parse::<f64>()
.ok()
.filter(|&v| v > 0.0 && v <= 1.0)
.ok_or_else(|| {
error::InvalidFulltextOptionSnafu {
msg: format!(
"Invalid false positive rate: {value}, expected: 0.0 < rate <= 1.0"
),
}
.build()
})?,
None => DEFAULT_FALSE_POSITIVE_RATE,
};
fulltext_options.false_positive_rate_in_10000 = (false_positive_rate * 10000.0) as u32;
}
Ok(fulltext_options)
}
}
@@ -638,23 +772,73 @@ impl fmt::Display for FulltextAnalyzer {
pub struct SkippingIndexOptions {
/// The granularity of the skip index.
pub granularity: u32,
/// The false positive rate of the skip index (in ten-thousandths, e.g., 100 = 1%).
#[serde(default = "index_options_default_false_positive_rate_in_10000")]
pub false_positive_rate_in_10000: u32,
/// The type of the skip index.
#[serde(default)]
pub index_type: SkippingIndexType,
}
impl SkippingIndexOptions {
/// Creates a new skipping index options without checking `false_positive_rate` and `granularity`.
pub fn new_unchecked(
granularity: u32,
false_positive_rate: f64,
index_type: SkippingIndexType,
) -> Self {
Self {
granularity,
false_positive_rate_in_10000: (false_positive_rate * 10000.0) as u32,
index_type,
}
}
/// Creates a new skipping index options.
pub fn new(
granularity: u32,
false_positive_rate: f64,
index_type: SkippingIndexType,
) -> Result<Self> {
ensure!(
0.0 < false_positive_rate && false_positive_rate <= 1.0,
error::InvalidSkippingIndexOptionSnafu {
msg: format!("Invalid false positive rate: {false_positive_rate}, expected: 0.0 < rate <= 1.0"),
}
);
ensure!(
granularity > 0,
error::InvalidSkippingIndexOptionSnafu {
msg: format!("Invalid granularity: {granularity}, expected: positive integer"),
}
);
Ok(Self::new_unchecked(
granularity,
false_positive_rate,
index_type,
))
}
/// Gets the false positive rate.
pub fn false_positive_rate(&self) -> f64 {
self.false_positive_rate_in_10000 as f64 / 10000.0
}
}
impl Default for SkippingIndexOptions {
fn default() -> Self {
Self {
granularity: DEFAULT_GRANULARITY,
index_type: SkippingIndexType::default(),
}
Self::new_unchecked(
DEFAULT_GRANULARITY,
DEFAULT_FALSE_POSITIVE_RATE,
SkippingIndexType::default(),
)
}
}
impl fmt::Display for SkippingIndexOptions {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
write!(f, "granularity={}", self.granularity)?;
write!(f, ", false_positive_rate={}", self.false_positive_rate())?;
write!(f, ", index_type={}", self.index_type)?;
Ok(())
}
@@ -681,15 +865,37 @@ impl TryFrom<HashMap<String, String>> for SkippingIndexOptions {
fn try_from(options: HashMap<String, String>) -> Result<Self> {
// Parse granularity with default value 1
let granularity = match options.get(COLUMN_SKIPPING_INDEX_OPT_KEY_GRANULARITY) {
Some(value) => value.parse::<u32>().map_err(|_| {
error::InvalidSkippingIndexOptionSnafu {
msg: format!("Invalid granularity: {value}, expected: positive integer"),
}
.build()
})?,
Some(value) => value
.parse::<u32>()
.ok()
.filter(|&v| v > 0)
.ok_or_else(|| {
error::InvalidSkippingIndexOptionSnafu {
msg: format!("Invalid granularity: {value}, expected: positive integer"),
}
.build()
})?,
None => DEFAULT_GRANULARITY,
};
// Parse false positive rate with default value 100
let false_positive_rate =
match options.get(COLUMN_SKIPPING_INDEX_OPT_KEY_FALSE_POSITIVE_RATE) {
Some(value) => value
.parse::<f64>()
.ok()
.filter(|&v| v > 0.0 && v <= 1.0)
.ok_or_else(|| {
error::InvalidSkippingIndexOptionSnafu {
msg: format!(
"Invalid false positive rate: {value}, expected: 0.0 < rate <= 1.0"
),
}
.build()
})?,
None => DEFAULT_FALSE_POSITIVE_RATE,
};
// Parse index type with default value BloomFilter
let index_type = match options.get(COLUMN_SKIPPING_INDEX_OPT_KEY_TYPE) {
Some(typ) => match typ.to_ascii_uppercase().as_str() {
@@ -704,10 +910,11 @@ impl TryFrom<HashMap<String, String>> for SkippingIndexOptions {
None => SkippingIndexType::default(),
};
Ok(SkippingIndexOptions {
Ok(SkippingIndexOptions::new_unchecked(
granularity,
false_positive_rate,
index_type,
})
))
}
}
@@ -973,4 +1180,59 @@ mod tests {
assert!(column_schema.default_constraint.is_none());
assert!(column_schema.metadata.is_empty());
}
#[test]
fn test_skipping_index_options_deserialization() {
let original_options = "{\"granularity\":1024,\"false-positive-rate-in-10000\":10,\"index-type\":\"BloomFilter\"}";
let options = serde_json::from_str::<SkippingIndexOptions>(original_options).unwrap();
assert_eq!(1024, options.granularity);
assert_eq!(SkippingIndexType::BloomFilter, options.index_type);
assert_eq!(0.001, options.false_positive_rate());
let options_str = serde_json::to_string(&options).unwrap();
assert_eq!(options_str, original_options);
}
#[test]
fn test_skipping_index_options_deserialization_v0_14_to_v0_15() {
let options = "{\"granularity\":10240,\"index-type\":\"BloomFilter\"}";
let options = serde_json::from_str::<SkippingIndexOptions>(options).unwrap();
assert_eq!(10240, options.granularity);
assert_eq!(SkippingIndexType::BloomFilter, options.index_type);
assert_eq!(DEFAULT_FALSE_POSITIVE_RATE, options.false_positive_rate());
let options_str = serde_json::to_string(&options).unwrap();
assert_eq!(options_str, "{\"granularity\":10240,\"false-positive-rate-in-10000\":100,\"index-type\":\"BloomFilter\"}");
}
#[test]
fn test_fulltext_options_deserialization() {
let original_options = "{\"enable\":true,\"analyzer\":\"English\",\"case-sensitive\":false,\"backend\":\"bloom\",\"granularity\":1024,\"false-positive-rate-in-10000\":10}";
let options = serde_json::from_str::<FulltextOptions>(original_options).unwrap();
assert!(!options.case_sensitive);
assert!(options.enable);
assert_eq!(FulltextBackend::Bloom, options.backend);
assert_eq!(FulltextAnalyzer::default(), options.analyzer);
assert_eq!(1024, options.granularity);
assert_eq!(0.001, options.false_positive_rate());
let options_str = serde_json::to_string(&options).unwrap();
assert_eq!(options_str, original_options);
}
#[test]
fn test_fulltext_options_deserialization_v0_14_to_v0_15() {
// 0.14 to 0.15
let options = "{\"enable\":true,\"analyzer\":\"English\",\"case-sensitive\":false,\"backend\":\"bloom\"}";
let options = serde_json::from_str::<FulltextOptions>(options).unwrap();
assert!(!options.case_sensitive);
assert!(options.enable);
assert_eq!(FulltextBackend::Bloom, options.backend);
assert_eq!(FulltextAnalyzer::default(), options.analyzer);
assert_eq!(DEFAULT_GRANULARITY, options.granularity);
assert_eq!(DEFAULT_FALSE_POSITIVE_RATE, options.false_positive_rate());
let options_str = serde_json::to_string(&options).unwrap();
assert_eq!(options_str, "{\"enable\":true,\"analyzer\":\"English\",\"case-sensitive\":false,\"backend\":\"bloom\",\"granularity\":10240,\"false-positive-rate-in-10000\":100}");
}
}

View File

@@ -12,6 +12,11 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use arrow_array::{
ArrayRef, PrimitiveArray, TimestampMicrosecondArray, TimestampMillisecondArray,
TimestampNanosecondArray, TimestampSecondArray,
};
use arrow_schema::DataType;
use common_time::timestamp::TimeUnit;
use common_time::Timestamp;
use paste::paste;
@@ -138,6 +143,41 @@ define_timestamp_with_unit!(Millisecond);
define_timestamp_with_unit!(Microsecond);
define_timestamp_with_unit!(Nanosecond);
pub fn timestamp_array_to_primitive(
ts_array: &ArrayRef,
) -> Option<(
PrimitiveArray<arrow_array::types::Int64Type>,
arrow::datatypes::TimeUnit,
)> {
let DataType::Timestamp(unit, _) = ts_array.data_type() else {
return None;
};
let ts_primitive = match unit {
arrow_schema::TimeUnit::Second => ts_array
.as_any()
.downcast_ref::<TimestampSecondArray>()
.unwrap()
.reinterpret_cast::<arrow_array::types::Int64Type>(),
arrow_schema::TimeUnit::Millisecond => ts_array
.as_any()
.downcast_ref::<TimestampMillisecondArray>()
.unwrap()
.reinterpret_cast::<arrow_array::types::Int64Type>(),
arrow_schema::TimeUnit::Microsecond => ts_array
.as_any()
.downcast_ref::<TimestampMicrosecondArray>()
.unwrap()
.reinterpret_cast::<arrow_array::types::Int64Type>(),
arrow_schema::TimeUnit::Nanosecond => ts_array
.as_any()
.downcast_ref::<TimestampNanosecondArray>()
.unwrap()
.reinterpret_cast::<arrow_array::types::Int64Type>(),
};
Some((ts_primitive, *unit))
}
#[cfg(test)]
mod tests {
use common_time::timezone::set_default_timezone;

View File

@@ -121,7 +121,9 @@ impl Default for FlownodeOptions {
logging: LoggingOptions::default(),
tracing: TracingOptions::default(),
heartbeat: HeartbeatOptions::default(),
query: QueryOptions::default(),
// flownode's query option is set to 1 to throttle flow's query so
// that it won't use too much cpu or memory
query: QueryOptions { parallelism: 1 },
user_provider: None,
}
}
@@ -903,7 +905,7 @@ impl StreamingEngine {
let rows_send = self.run_available(true).await?;
let row = self.send_writeback_requests().await?;
debug!(
"Done to flush flow_id={:?} with {} input rows flushed, {} rows sended and {} output rows flushed",
"Done to flush flow_id={:?} with {} input rows flushed, {} rows sent and {} output rows flushed",
flow_id, flushed_input_rows, rows_send, row
);
Ok(row)

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