Compare commits

...

93 Commits

Author SHA1 Message Date
Eugene Tolbakov
ca4d690424 feat: add modulo function (#3147)
* feat: add modulo function

* fix: address CR feedback
2024-01-13 00:24:25 +00:00
Weny Xu
75975adcb6 fix: fix tests failed on windows (#3155)
* fix: fix tests failed on windows

* feat: add comments

* Update src/object-store/src/util.rs

Co-authored-by: Yingwen <realevenyag@gmail.com>

---------

Co-authored-by: Yingwen <realevenyag@gmail.com>
2024-01-12 09:12:11 +00:00
Ruihang Xia
527e523a38 fix: handle non-identical time index and field column in PromQL set operation (#3145)
* handle different field columns

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

* fix and/unless on different time index

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

* update sqlness result

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

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2024-01-12 06:27:03 +00:00
Weny Xu
aad2afd3f2 chore: bump version to 0.6.0 (#3154) 2024-01-12 06:25:14 +00:00
Weny Xu
bf88b3b4a0 fix: fix store all wal options (#3149)
* fix: fix store all wal options

* fix: incorrect updating DatanodeTable value
2024-01-12 04:48:14 +00:00
Weny Xu
bf96ce3049 fix: print detailed error (#3146) 2024-01-12 04:02:32 +00:00
Weny Xu
430ffe0e28 fix(kafka): overwrite the EntryId with Offset while consuming records (#3148)
* fix(kafka): overwrite the EntryId with Offset while consuming the KafkaRecords

* fix: temporarily workaround of incorrect entry Id
2024-01-12 03:46:17 +00:00
Zhenchi
c1190bae7b feat(mito): support write cache for index file (#3144)
* feat(mito): support write cache for index file

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

* fix: address comments

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

* fix: merge main

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

---------

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>
2024-01-12 02:40:56 +00:00
Ruihang Xia
0882da4d01 feat: support PromQL operations over the same metric (#3124)
* update sqlness result

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

* update ut cases

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

* remove deadcode

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

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2024-01-11 23:07:17 +00:00
Wei
8ec1e42754 feat: read data from write cache (#3128)
* feat: read from write cache

* chore: add read ranges test

* fix: use get instead of contains_key

* chore: clippy

* chore: cr comment

Co-authored-by: Yingwen <realevenyag@gmail.com>

* fix: with_label_values

---------

Co-authored-by: Yingwen <realevenyag@gmail.com>
2024-01-11 12:06:28 +00:00
Ruihang Xia
b00b49284e feat: manager kafka cluster in sqlness runner (#3143)
* feat: manager kafka cluster in sqlness runner

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

* pull up clippy config

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

* Apply suggestions from code review

Co-authored-by: niebayes <niebayes@gmail.com>

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
Co-authored-by: niebayes <niebayes@gmail.com>
2024-01-11 09:47:19 +00:00
Ruihang Xia
09b3c7029b feat: handle drop request for metric table (#3136)
* handle drop request

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

* adjust procedure manager

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

* add create table sqlness test

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

* insert/query metric table

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

* address CR comments

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

* Update src/common/meta/src/kv_backend.rs

Co-authored-by: JeremyHi <jiachun_feng@proton.me>

* fix clippy

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

* reuse region option for metadata region

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

* tweak variable name

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

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
Co-authored-by: JeremyHi <jiachun_feng@proton.me>
2024-01-11 09:38:43 +00:00
niebayes
f5798e2833 fix: remove incorrect wal comments in config file (#3142)
fix: kafka config comments
2024-01-11 09:34:24 +00:00
Zhenchi
fd8fb641fd feat(parquet): introduce inverted index applier to reader (#3130)
* feat(parquet): introduce inverted index applier to reader

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

* feat: purger removes index file

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

* fix test

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

* chore: add TODO for escape route

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

* chore: add TODO for escape route

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

* Update src/mito2/src/access_layer.rs

Co-authored-by: dennis zhuang <killme2008@gmail.com>

* Update src/mito2/src/sst/parquet/reader.rs

Co-authored-by: dennis zhuang <killme2008@gmail.com>

* feat: min-max index to prune row groups filtered by inverted index

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

* feat: file_meta.inverted_index_available -> file_meta.available_indexes

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

* chore: add TODO for leveraging WriteCache

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

* fix fmt

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

* fix: misset available indexes

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

* feat: add index file size

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

* refactor: use smallvec to reduce heap allocation

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

* fix: add index size to disk usage

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

---------

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>
Co-authored-by: dennis zhuang <killme2008@gmail.com>
2024-01-11 08:04:59 +00:00
Weny Xu
312e8e824e fix: save code in debug_assert! (#3137)
fix: save code in debug_assert!
2024-01-11 06:07:08 +00:00
Wei
29a7f301df feat: write and upload sst (#3106)
* feat: write and upload sst file

* refactor: unit test

* cr comment

* chore: typos

* chore: cr comment

* chore: conflict

* Apply suggestions from code review

Co-authored-by: dennis zhuang <killme2008@gmail.com>

* chore: fmt

* chore: style

Co-authored-by: Yingwen <realevenyag@gmail.com>

---------

Co-authored-by: dennis zhuang <killme2008@gmail.com>
Co-authored-by: Yingwen <realevenyag@gmail.com>
2024-01-11 02:34:16 +00:00
LFC
51a3fbc7bf refactor: change how frontend grpc services are orchestrated (#3134) 2024-01-11 02:26:44 +00:00
Lanqing Yang
d521bc9dc5 chore: impl KvBackend for MetaPeerClient (#3076) 2024-01-10 14:16:03 +00:00
Weny Xu
7fad4e8356 fix: incorrect parsing broker_endpoints env variable (#3135) 2024-01-10 13:59:49 +00:00
Ning Sun
b6033f62cd refactor: implement version as built-in function and use fixed mysql version (#3133)
* refactor:  implement version as built-in function

* test: add sqlness test for version()
2024-01-10 11:04:18 +00:00
dennis zhuang
fd3f23ea15 feat: adds runtime_metrics (#3127)
* feat: adds runtime_metrics

* fix: comment

* feat: refactor metrics table

* chore: ensure build_info and runtime_metrics only avaiable in greptime catalog

* feat: adds timestamp column
2024-01-10 10:51:30 +00:00
niebayes
1b0e39a7f2 chore: stop exposing num_partitions (#3132) 2024-01-10 10:45:18 +00:00
Weny Xu
3ab370265a feat: expose the region migration replay_timeout argument (#3129)
* feat: expose region migration args

* fix: fix ci
2024-01-10 09:47:59 +00:00
Weny Xu
ec8266b969 refactor: refactor the locks in the procedure (#3126)
* feat: add lock key

* refactor: procedure lock keys

* chore: apply suggestions from CR
2024-01-10 09:46:39 +00:00
Zhenchi
490312bf57 fix: unstable time record test (#3131)
Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>
2024-01-10 09:41:52 +00:00
Ning Sun
1fc168bf6a feat: update our cross schema check to cross catalog (#3123) 2024-01-09 09:38:48 +00:00
Zhenchi
db98484796 feat(inverted_index): introduce SstIndexCreator (#3107)
* feat(inverted_index): introduce SstIndexCreator

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

* chore: tiny polish

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

* feat: distinguish intermediate store and index store

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

* chore: move comment as doc comment

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

* refactor: column id as index name

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

---------

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>
2024-01-09 09:24:16 +00:00
Ruihang Xia
7d0d2163d2 fix: expose unsupported datatype error on mysql protocol (#3121)
Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2024-01-09 09:13:53 +00:00
Ruihang Xia
c4582c05cc chore: change the default doc checkbox to no need (#3122)
Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2024-01-09 17:12:54 +08:00
niebayes
a0a31c8acc chore(remote_wal): remove topic alias (#3120)
chore: remove topic alias
2024-01-09 07:35:02 +00:00
tison
0db1861452 chore(python): Print Python interpreter version (#3118)
* chore(pyo3_backend): Print bundle Python interpreter version

Signed-off-by: tison <wander4096@gmail.com>

* print RustPython interpreter version on init

Signed-off-by: tison <wander4096@gmail.com>

---------

Signed-off-by: tison <wander4096@gmail.com>
2024-01-09 07:04:23 +00:00
Wei
225ae953d1 feat: add parquet metadata to cache (#3097)
* feat: parquet metadata to sst meta cache

* chore: clippy

* refactor: move code to access_layer

* chore: clone()
2024-01-09 07:00:42 +00:00
Lei, HUANG
2c1b1cecc8 chore: add bound check for raft-engine logstore (#3073)
* chore: add bound check for raft-engine logstore

* feat: add bound check to append_batch API

* chore: check entry id during replay

* chore: resolve conflicts

* feat: add allow_stale_entries options to force obsolete wal entries

* chore: resolve some comments
2024-01-09 06:42:46 +00:00
Lei, HUANG
62db28b465 feat: add options to enable log recycle and periodical fsync (#3114)
* feat: add options to enable log recycle and periodical fsync

* fix: resolve review comments

* fix: conflicts
2024-01-09 06:41:23 +00:00
fys
6e860bc0fd feat: support grpc for otlp trace and metrics (#3105)
* feat: add grpc support for otlp trace and metrics

* cr: add some comment

* fix: ut

* fix: cr
2024-01-09 05:01:48 +00:00
Yingwen
8bd4a36136 feat(mito): Init the write cache in datanode (#3100)
* feat: add builder to build cache manager

* refactor: make MitoEngine::new async

* refactor: refactor object store creation

* refactor: add helper fn to attaches layers

* feat: fn to build fs store

* feat: add write cache to engine

* feat: config write cache

* style: fix clippy

* test: fix test

* feat: add warning

* chore: add experimental prefix to configs

* test: fix config test

* test: test weighted size

* feat: add switch to enable write cache

* fix: update cache stats by using get

* style: use then
2024-01-09 04:40:22 +00:00
Ruihang Xia
af0c4c068a feat: support PromQL function vector (#3036)
* produce vector plan

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

* work with OR

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

* apply review sugg

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

* move common const strings to common_query

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

* add comment for GREPTIME_COUNT

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

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2024-01-09 03:44:00 +00:00
dennis zhuang
26cbcb8b3a docs: update issue template (#3119) 2024-01-09 02:45:55 +00:00
niebayes
122b47210e chore: bump version to 0.5.1 (#3116) 2024-01-08 11:32:56 +00:00
tison
316d843482 feat: support CSV format in sql HTTP API (#3062)
* chore: fix typo

Signed-off-by: tison <wander4096@gmail.com>

* add csv format

Signed-off-by: tison <wander4096@gmail.com>

* flatten response

Signed-off-by: tison <wander4096@gmail.com>

* more flatten response

Signed-off-by: tison <wander4096@gmail.com>

* add CSV format

Signed-off-by: tison <wander4096@gmail.com>

* format InfluxdbV1Response

Signed-off-by: tison <wander4096@gmail.com>

* format ErrorResponse

Signed-off-by: tison <wander4096@gmail.com>

* propagate ErrorResponse to InfluxdbV1Response

Signed-off-by: tison <wander4096@gmail.com>

* format GreptimedbV1Response

Signed-off-by: tison <wander4096@gmail.com>

* format CsvResponse

Signed-off-by: tison <wander4096@gmail.com>

* impl IntoResponse for QueryResponse

Signed-off-by: tison <wander4096@gmail.com>

* promql

Signed-off-by: tison <wander4096@gmail.com>

* sql

Signed-off-by: tison <wander4096@gmail.com>

* compile

Signed-off-by: tison <wander4096@gmail.com>

* fixup aide

Signed-off-by: tison <wander4096@gmail.com>

* clear debt

Signed-off-by: tison <wander4096@gmail.com>

* fixup UT test_recordbatches_conversion

Signed-off-by: tison <wander4096@gmail.com>

* fixup IT cases

Signed-off-by: tison <wander4096@gmail.com>

* fixup more IT cases

Signed-off-by: tison <wander4096@gmail.com>

* fixup test-integration cases

Signed-off-by: tison <wander4096@gmail.com>

* update comment

Signed-off-by: tison <wander4096@gmail.com>

* fixup deserialize and most query < 1ms

Signed-off-by: tison <wander4096@gmail.com>

* fixup auth tests

Signed-off-by: tison <wander4096@gmail.com>

* fixup tests

Signed-off-by: tison <wander4096@gmail.com>

* fixup and align X-GreptimeDB headers

Signed-off-by: tison <wander4096@gmail.com>

* fixup compile

Signed-off-by: tison <wander4096@gmail.com>

---------

Signed-off-by: tison <wander4096@gmail.com>
2024-01-08 10:54:27 +00:00
niebayes
8c58d3f85b test(remote_wal): add unit tests for kafka remote wal (#2993)
* test: add unit tests

* feat: introduce kafka runtime backed by testcontainers

* test: add test for kafka runtime

* fix: format

* chore: make kafka image ready to be used

* feat: add entry builder

* tmp

* test: add unit tests for client manager

* test: add some unit tests for kafka log store

* chore: resolve some todos

* chore: resolve some todos

* test: add unit tests for kafka log store

* chore: add deprecate develop branch warning

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

* tmp: ready to move unit tests to an indie dir

* test: update unit tests for client manager

* test: add unit tests for meta srv remote wal

* fix: license

* fix: test

* refactor: kafka image

* doc: add doc example for kafka image

* chore: migrate kafka image to an indie PR

* fix: CR

* fix: CR

* fix: test

* fix: CR

* fix: update Cargo.toml

* fix: CR

* feat: skip test if no endpoints env

* fix: format

* test: rewrite parallel test with barrier

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
Co-authored-by: Ruihang Xia <waynestxia@gmail.com>
2024-01-08 10:48:11 +00:00
LFC
fcacb100a2 chore: expose some codes to let other projects use them (#3115) 2024-01-08 06:32:01 +00:00
Weny Xu
58ada1dfef fix: check env before running kafka test (#3110)
* fix: check env before running kafka test

* Apply suggestions from code review

Co-authored-by: niebayes <niebayes@gmail.com>

---------

Co-authored-by: niebayes <niebayes@gmail.com>
2024-01-08 06:30:43 +00:00
Weny Xu
f78c467a86 chore: bump opendal to 0.44.1 (#3111) 2024-01-08 03:55:58 +00:00
niebayes
78303639db feat(remote_wal): split an entry if it's too large (#3092)
* feat: split an entry if it's too large

* chore: rewrite check records

* test: add some unit tests for record

* chore: rewrite entry splitting

* chore: add unit tests for build records

* chore: add more unit tests for record

* chore: rewrite encdec of record

* revert: ignored test

* fix: set limit for max_batch_size

* fix: clippy

* chore: remove heavy logging

* fix: CR

* fix: properly terminate

* fix: CR

* fix: compiling

* fix: sqlness

* fix: CR

* fix: license

* fix: license
2024-01-05 12:41:43 +00:00
JeremyHi
bd1a5dc265 feat: metric engine support alter (#3098)
* feat: metric engine support alter

* chore: by comment

* feat: get physical table route for frontend
2024-01-05 09:46:39 +00:00
Weny Xu
e0a43f37d7 chore: bump opendal to 0.44 (#3058)
* chore: bump opendal to 0.44

* fix: fix test_object_store_cache_policy

* Revert "fix: fix test_object_store_cache_policy"

This reverts commit 46c37c343f66114e0f6ee7a0a3b9ee2b79c810af.

* fix: fix test_object_store_cache_policy

* fix: fix test_file_backend_with_lru_cache

* chore: apply suggestions from CR

* fix(mito): fix mito2 cache

* chore: apply suggestions from CR

* chore: apply suggestions from CR
2024-01-05 09:05:41 +00:00
zyy17
a89840f5f9 refactor(metrics): add 'greptime_' prefix for every metrics (#3093)
* refactor(metrics): add 'greptimedb_' prefix for every metrics

* chore: use 'greptime_' as prefix

* chore: add some prefix for new metrics

* chore: fix format error
2024-01-05 08:12:23 +00:00
dennis zhuang
c2db970687 feat: pushdown filters for some information_schema tables (#3091)
* feat: pushdown scan request to information_schema tables stream

* feat: supports filter pushdown for columns

* feat: supports filter pushdown for some information_schema tables

* fix: typo

* fix: predicate evaluate

* fix: typo

* test: predicates

* fix: comment

* fix: pub mod

* docs: improve comments

* fix: cr comments and supports like predicate

* chore: typo

* fix: cargo toml format

* chore: apply suggestion
2024-01-05 07:18:22 +00:00
LFC
e0525dbfeb chore: expose some codes to let other projects use them (#3102) 2024-01-05 06:54:01 +00:00
Weny Xu
cdc9021160 feat(metric): implement role and region_disk_usage (#3095)
* feat(metric): implement `role` and `region_disk_usage`

* Update src/datanode/src/region_server.rs

* Update src/datanode/src/heartbeat.rs

---------

Co-authored-by: LFC <990479+MichaelScofield@users.noreply.github.com>
2024-01-05 06:53:52 +00:00
dennis zhuang
702ea32538 docs: update the description of greptimedb project (#3099)
* docs: update the info of greptimedb project

* chore: move up SQL/PromQL
2024-01-05 03:06:02 +00:00
Weny Xu
342faa4e07 test: add tests for lease keeper with logical table (#3096) 2024-01-05 02:29:48 +00:00
tison
44ba131987 fix: improve redact sql regexp (#3080)
Signed-off-by: tison <wander4096@gmail.com>
2024-01-04 14:53:20 +00:00
Yingwen
96b6235f25 feat(mito): Add WriteCache struct and write SSTs to write cache (#2999)
* docs: remove todo

* feat: add upload cache

* feat: add cache to sst write path

* feat: add storage to part

* feat: add dir to part

* feat: revert storage name

* feat: flush use upload part writer

* feat: use upload part writer in compaction task

* refactor: upload part writer builds parquet writer

* chore: suppress warnings

* refactor: rename UploadCache to WriteCache

* refactor: move source to write_all()

* chore: typos

* chore: remove output mod

* feat: changes upload to async method

* docs: update cache

* chore: fix compiler errors

* docs: remove comment

* chore: simplify upload part

* refactor: remove option from cache manager param to access layer

* feat: remove cache home from file cache

* feat: write cache holds file cache

* feat: add recover and pub some methods

* feat: remove usages of UploadPartWriter

* refactor: move sst_file_path to sst mod

* refactor: use write cache in access layer

* refactor: remove upload

* style: fix clippy

* refactor: pub write cache method/structs
2024-01-04 10:53:43 +00:00
Weny Xu
f1a4750576 feat(tests-integration): add more region migration integration tests (#3094) 2024-01-04 08:18:46 +00:00
Zhenchi
d973cf81f0 feat(inverted_index): implement apply for SstIndexApplier (#3088)
* feat(inverted_index): implement apply for SstIndexApplier

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

* chore: rename metrics

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

---------

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>
2024-01-04 07:33:03 +00:00
Weny Xu
284a496f54 feat: add logs for upgrading candidate region and updating metadata (#3077)
* feat: add logs for upgrading candidate region

* feat: add logs for update metadata

* chore: apply suggestions from CR
2024-01-04 06:57:07 +00:00
WU Jingdi
4d250ed054 fix: Optimize export metric behavior (#3047)
* fix: optimze export metric bahavior

* chor: fix ci

* chore: update config format

* chore: fix format
2024-01-04 06:40:50 +00:00
LFC
ec43b9183d feat: table route for metric engine (#3053)
* feat: table route for metric engine

* feat: register logical regions

* fix: open logical region (#96)

---------

Co-authored-by: JeremyHi <jiachun_feng@proton.me>
2024-01-04 06:30:17 +00:00
ZonaHe
b025bed45c feat: update dashboard to v0.4.6 (#3089)
Co-authored-by: ZonaHex <ZonaHex@users.noreply.github.com>
2024-01-04 02:56:41 +00:00
Weny Xu
21694c2a1d feat: abort region migration if leader region peer is unexpected (#3086) 2024-01-03 11:46:51 +00:00
ClSlaid
5c66ce6e88 chore: remove unnecessary result wrappings (#3084)
patch: remove unnecessary result wrappings

Signed-off-by: 蔡略 <cailue@bupt.edu.cn>
2024-01-03 10:20:33 +00:00
Weny Xu
b2b752337b fix: fix non-physical error msg (#3087) 2024-01-03 09:40:03 +00:00
Weny Xu
aa22f9c94a refactor: allow procedure to acquire share lock (#3061)
* feat: implement `KeyRwLock`

* refactor: use KeyRwLock instead of LockMap

* refactor: use StringKey instead of String

* chore: remove redundant code

* refactor: cleanup KeyRwLock staled locks before granting new lock

* feat: clean staled locks manually

* feat: sort lock key in lexicographically order

* feat: ensure the ref count before dropping the rwlock

* feat: add more tests for rwlock

* feat: drop the key guards first

* feat: drops the key guards in the reverse order

* chore: apply suggestions from CR

* chore: apply suggestions from CR

* chore: apply suggestions from CR
2024-01-03 08:05:45 +00:00
Weny Xu
611a8aa2fe feat(tests-integration): add a naive region migration integration test (#3078)
* fix: fix heartbeat handler ignore upgrade candidate instruction

* fix: fix handler did not inject wal options

* feat: expose `RegionMigrationProcedureTask`

* feat(tests-integration): add a naive region migration test

* chore: apply suggestions from CR

* feat: add test if the target region has migrated

* chore: apply suggestions from CR
2024-01-03 07:12:59 +00:00
Zhenchi
e4c71843e6 feat(inverted_index): get memory usage of appliers (#3081)
Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>
2024-01-03 06:56:56 +00:00
Zhenchi
e1ad7af10c feat(puffin): finish return written bytes (#3082)
Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>
2024-01-03 06:55:09 +00:00
Zhenchi
b9302e4f0d feat(inverted_index): Add applier builder to convert Expr to Predicates (Part 2) (#3068)
* feat(inverted_index.integration): Add applier builder to convert Expr to Predicates (Part 1)

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

* feat(inverted_index.integration): Add applier builder to convert Expr to Predicates (Part 2)

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

* test: add comparison unit tests

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

* test: add eq_list unit tests

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

* test: add in_list unit tests

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

* test: add and unit tests

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

* test: strip tests

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

* fix: address comments

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

---------

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>
2024-01-03 05:14:40 +00:00
Yingwen
2e686fe053 feat(mito): Implement file cache (#3022)
* feat: recover cache

* feat: moka features

* test: tests for file cache

* chore: suppress warninig

* fix: parse_inde_key consider suffix

* feat: update cache

* feat: expose cache file path

* feat: use cache_path in test
2024-01-03 02:05:06 +00:00
Weny Xu
128d3717fa test(tests-integration): add a naive test with kafka wal (#3071)
* chore(tests-integration): add setup tests with kafka wal to README.md

* feat(tests-integration): add meta wal config

* fix(tests-integration): fix sign of both_instances_cases_with_kafka_wal

* chore(tests-integration): set num_topic to 3 for tests

* test(tests-integration): add a naive test with kafka wal

* chore: apply suggestions from CR
2024-01-02 09:05:20 +00:00
Weny Xu
2b181e91e0 refactor: unify the injection of WAL option (#3066)
* feat: add prepare_wal_option

* refactor: use integer hashmap

* feat: unify the injection of WAL option

* fix: fix procedure_flow_upgrade_candidate_with_retry

* chore: apply suggestions from CR
2024-01-02 07:40:02 +00:00
Weny Xu
d87ab06b28 feat: add kafka wal integration test utils (#3069)
* feat(tests-integration): add wal_config

* feat: add kafka wal integration test utils
2024-01-02 07:38:43 +00:00
Weny Xu
5653389063 feat!: correct the kafka config option (#3065)
* feat: correct the kafka config option

* refactor: rewrite the verbose comments
2024-01-02 07:31:37 +00:00
dimbtp
c4d7b0d91d feat: add some tables for information_schema (#3060)
* feat: add information_schema.optimizer_trace

* feat: add information_schema.parameters

* feat: add information_schema.profiling

* feat: add information_schema.referential_constraints

* feat: add information_schema.routines

* feat: add information_schema.schema_privileges

* feat: add information_schema.table_privileges

* feat: add information_schema.triggers

* fix: update sql test result

* feat: add information_schema.global_status

* feat: add information_schema.session_status

* fix: update sql test result

* fix: add TODO for some tables

* Update src/catalog/src/information_schema/memory_table/tables.rs

Co-authored-by: Yingwen <realevenyag@gmail.com>

---------

Co-authored-by: dennis zhuang <killme2008@gmail.com>
Co-authored-by: Yingwen <realevenyag@gmail.com>
2024-01-02 04:10:59 +00:00
dimbtp
f735f739e5 feat: add information_schema.key_column_usage (#3057)
* feat: add information_schema.key_column_usage

* fix: follow #3057 review comments

* fix: add sql test for `key_column_usage` table

* fix: fix spell typo

* fix: resolve conflict in sql test result
2023-12-31 12:29:06 +00:00
dimbtp
6070e88077 feat: add information_schema.files (#3054)
* feat: add information_schema.files

* fix: update information_schema.result

* fix: change `EXTRA` field type to string
2023-12-31 02:08:16 +00:00
niebayes
9db168875c fix(remote_wal): some known issues (#3052)
* fix: some known issues

* fix: CR

* fix: CR

* chore: replace Mutex with RwLock
2023-12-30 15:28:10 +00:00
AntiTopQuark
4460af800f feat(TableRouteValue): add panic notes and type checks (#3031)
* refactor(TableRouteValue): add panic notes and type checks

* chore: add deprecate develop branch warning

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

* add error defines and checks

* Update README.md

* update code format and fix tests

* update name of error

* delete unused note

* fix unsafe .expect() for region_route()

* update error name

* update unwrap

* update code format

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
Co-authored-by: Ruihang Xia <waynestxia@gmail.com>
2023-12-30 13:02:26 +00:00
Zhenchi
69a53130c2 feat(inverted_index): Add applier builder to convert Expr to Predicates (Part 1) (#3034)
* feat(inverted_index.integration): Add applier builder to convert Expr to Predicates (Part 1)

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

* chore: add docs

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

* fix: typos

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

* fix: address comments

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

* Update src/mito2/src/sst/index/applier/builder.rs

Co-authored-by: Yingwen <realevenyag@gmail.com>

* fix: remove unwrap

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

* chore: error source

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

---------

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>
Co-authored-by: Yingwen <realevenyag@gmail.com>
2023-12-30 07:32:32 +00:00
Ning Sun
1c94d4c506 ci: fix duplicatd doc issue (#3056) 2023-12-30 13:36:14 +08:00
Ning Sun
41e51d4ab3 chore: attempt to add doc issue in label task (#3021)
* chore: attempt to add doc issue in label task

* ci: check pr body for doc issue creation
2023-12-29 20:17:34 +08:00
dennis zhuang
11ae85b1cd feat: adds information_schema.schemata (#3051)
* feat: improve information_schema.columns

* feat: adds information_schema.schemata

* fix: instance test

* fix: comment
2023-12-29 09:22:31 +00:00
LFC
7551432cff refactor: merge standalone and metasrv table metadata allocators (#3035)
* refactor: merge standalone and metasrv table metadata allocators

* Update src/common/meta/src/ddl/table_meta.rs

Co-authored-by: niebayes <niebayes@gmail.com>

* Update src/common/meta/src/ddl/table_meta.rs

Co-authored-by: Weny Xu <wenymedia@gmail.com>

---------

Co-authored-by: niebayes <niebayes@gmail.com>
Co-authored-by: Weny Xu <wenymedia@gmail.com>
2023-12-29 08:50:59 +00:00
Weny Xu
e16f093282 test(remote_wal): add sqlness with kafka wal (#3027)
* feat(sqlness): add kafka wal config

* chore: add sqlness with kafka wal ci config

* fix: fix config

* chore: apply suggestions from CR

* fix: add metasrv config to sqlness with kafka

* fix: replay memtable should from flushed_entry_id + 1

* fix: should set append flag to fopen

* feat: start wal allocator in standalone meta mode

* feat: append a noop record after kafka topic initialization

* test: ignore tests temporally

* test: change sqlness kafka wal config
2023-12-29 08:17:22 +00:00
Weny Xu
301ffc1d91 feat(remote_wal): append a noop record after kafka topic initialization (#3040)
* feat: append a noop record after kafka topic initialization

* chore: apply suggestions from CR

* feat: ignore the noop record during the read
2023-12-29 07:46:48 +00:00
Weny Xu
d22072f68b feat: expose region migration http endpoint (#3032)
* feat: add region migration endpoint

* feat: implement naive peer registry

* chore: apply suggestions from CR

* chore: rename `ContextFactoryImpl` to `DefaultContextFactory`

* chore: rename unregister to deregister

* refactor: use lease-based alive datanode checking
2023-12-29 06:57:00 +00:00
Weny Xu
b526d159c3 fix: replay memtable should from flushed_entry_id + 1 (#3038)
* fix: replay memtable should from flushed_entry_id + 1

* chore: apply suggestions from CR
2023-12-28 16:12:07 +00:00
ZonaHe
7152407428 feat: update dashboard to v0.4.5 (#3033)
Co-authored-by: ZonaHex <ZonaHex@users.noreply.github.com>
2023-12-28 11:51:43 +00:00
Ruihang Xia
b58296de22 feat: Implement OR for PromQL (#3024)
* with anit-join

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

* impl UnionDistinctOn

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

* unify schema

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

* fix clippy

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

* add sqlness case

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

* add UTs

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

* Update src/promql/src/planner.rs

Co-authored-by: dennis zhuang <killme2008@gmail.com>

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
Co-authored-by: dennis zhuang <killme2008@gmail.com>
2023-12-28 06:56:17 +00:00
Yingwen
1d80a0f2d6 chore: Update CI badge in README.md (#3028)
chore: Update README.md

Fix CI badge
2023-12-28 05:59:27 +00:00
Ruihang Xia
286b9af661 chore: change all reference from develop to main (#3026)
Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2023-12-28 04:11:00 +00:00
dennis zhuang
af13eeaad3 feat: adds character_sets, collations and events etc. (#3017)
feat: adds character_sets, collations and events etc. to information_schema
2023-12-28 04:01:42 +00:00
357 changed files with 17690 additions and 3927 deletions

View File

@@ -19,3 +19,5 @@ GT_GCS_BUCKET = GCS bucket
GT_GCS_SCOPE = GCS scope
GT_GCS_CREDENTIAL_PATH = GCS credential path
GT_GCS_ENDPOINT = GCS end point
# Settings for kafka wal test
GT_KAFKA_ENDPOINTS = localhost:9092

View File

@@ -21,6 +21,7 @@ body:
- Locking issue
- Performance issue
- Unexpected error
- User Experience
- Other
validations:
required: true
@@ -33,9 +34,14 @@ body:
multiple: true
options:
- Standalone mode
- Distributed Cluster
- Storage Engine
- Query Engine
- Table Engine
- Write Protocols
- MetaSrv
- Frontend
- Datanode
- Meta
- Other
validations:
required: true
@@ -77,6 +83,17 @@ body:
validations:
required: true
- type: input
id: greptimedb
attributes:
label: What version of GreptimeDB did you use?
description: |
Please provide the version of GreptimeDB. For example:
0.5.1 etc. You can get it by executing command line `greptime --version`.
placeholder: "0.5.1"
validations:
required: true
- type: textarea
id: logs
attributes:

View File

@@ -15,6 +15,6 @@ Please explain IN DETAIL what the changes are in this PR and why they are needed
- [ ] I have written the necessary rustdoc comments.
- [ ] I have added the necessary unit tests and integration tests.
- [ ] This PR does not require documentation updates.
- [x] This PR does not require documentation updates.
## Refer to a related PR or issue link (optional)

View File

@@ -1,7 +1,7 @@
on:
push:
branches:
- develop
- main
paths-ignore:
- 'docs/**'
- 'config/**'

View File

@@ -11,7 +11,6 @@ on:
- '.gitignore'
push:
branches:
- develop
- main
paths-ignore:
- 'docs/**'
@@ -105,6 +104,37 @@ jobs:
path: ${{ runner.temp }}/greptime-*.log
retention-days: 3
sqlness-kafka-wal:
name: Sqlness Test with Kafka Wal
if: github.event.pull_request.draft == false
runs-on: ${{ matrix.os }}
strategy:
matrix:
os: [ ubuntu-20.04-8-cores ]
timeout-minutes: 60
steps:
- uses: actions/checkout@v3
- uses: arduino/setup-protoc@v1
with:
repo-token: ${{ secrets.GITHUB_TOKEN }}
- uses: dtolnay/rust-toolchain@master
with:
toolchain: ${{ env.RUST_TOOLCHAIN }}
- name: Rust Cache
uses: Swatinem/rust-cache@v2
- name: Setup kafka server
working-directory: tests-integration/fixtures/kafka
run: docker compose -f docker-compose-standalone.yml up -d --wait
- name: Run sqlness
run: cargo sqlness -w kafka -k 127.0.0.1:9092
- name: Upload sqlness logs
if: always()
uses: actions/upload-artifact@v3
with:
name: sqlness-logs
path: ${{ runner.temp }}/greptime-*.log
retention-days: 3
fmt:
name: Rustfmt
if: github.event.pull_request.draft == false

View File

@@ -18,3 +18,14 @@ jobs:
enable-versioned-regex: false
repo-token: ${{ secrets.GITHUB_TOKEN }}
sync-labels: 1
- name: create an issue in doc repo
uses: dacbd/create-issue-action@main
if: ${{ github.event.action == 'opened' && contains(github.event.pull_request.body, '- [ ] This PR does not require documentation updates.') }}
with:
owner: GreptimeTeam
repo: docs
token: ${{ secrets.DOCS_REPO_TOKEN }}
title: Update docs for ${{ github.event.issue.title || github.event.pull_request.title }}
body: |
A document change request is generated from
${{ github.event.issue.html_url || github.event.pull_request.html_url }}

View File

@@ -11,7 +11,6 @@ on:
- '.gitignore'
push:
branches:
- develop
- main
paths:
- 'docs/**'

View File

@@ -3,7 +3,7 @@ name: License checker
on:
push:
branches:
- develop
- main
pull_request:
types: [opened, synchronize, reopened, ready_for_review]
jobs:

View File

@@ -91,7 +91,7 @@ env:
# The scheduled version is '${{ env.NEXT_RELEASE_VERSION }}-nightly-YYYYMMDD', like v0.2.0-nigthly-20230313;
NIGHTLY_RELEASE_PREFIX: nightly
# Note: The NEXT_RELEASE_VERSION should be modified manually by every formal release.
NEXT_RELEASE_VERSION: v0.6.0
NEXT_RELEASE_VERSION: v0.7.0
jobs:
allocate-runners:

View File

@@ -10,7 +10,7 @@ Follow our [README](https://github.com/GreptimeTeam/greptimedb#readme) to get th
It can feel intimidating to contribute to a complex project, but it can also be exciting and fun. These general notes will help everyone participate in this communal activity.
- Follow the [Code of Conduct](https://github.com/GreptimeTeam/greptimedb/blob/develop/CODE_OF_CONDUCT.md)
- Follow the [Code of Conduct](https://github.com/GreptimeTeam/greptimedb/blob/main/CODE_OF_CONDUCT.md)
- Small changes make huge differences. We will happily accept a PR making a single character change if it helps move forward. Don't wait to have everything working.
- Check the closed issues before opening your issue.
- Try to follow the existing style of the code.
@@ -26,7 +26,7 @@ Pull requests are great, but we accept all kinds of other help if you like. Such
## Code of Conduct
Also, there are things that we are not looking for because they don't match the goals of the product or benefit the community. Please read [Code of Conduct](https://github.com/GreptimeTeam/greptimedb/blob/develop/CODE_OF_CONDUCT.md); we hope everyone can keep good manners and become an honored member.
Also, there are things that we are not looking for because they don't match the goals of the product or benefit the community. Please read [Code of Conduct](https://github.com/GreptimeTeam/greptimedb/blob/main/CODE_OF_CONDUCT.md); we hope everyone can keep good manners and become an honored member.
## License

168
Cargo.lock generated
View File

@@ -196,7 +196,7 @@ checksum = "8f1f8f5a6f3d50d89e3797d7593a50f96bb2aaa20ca0cc7be1fb673232c91d72"
[[package]]
name = "api"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"common-base",
"common-decimal",
@@ -674,7 +674,7 @@ dependencies = [
[[package]]
name = "auth"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"api",
"async-trait",
@@ -847,7 +847,7 @@ dependencies = [
[[package]]
name = "benchmarks"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"arrow",
"chrono",
@@ -1179,10 +1179,11 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5"
[[package]]
name = "catalog"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"api",
"arc-swap",
"arrow",
"arrow-schema",
"async-stream",
"async-trait",
@@ -1205,6 +1206,7 @@ dependencies = [
"datatypes",
"futures",
"futures-util",
"itertools 0.10.5",
"lazy_static",
"log-store",
"meta-client",
@@ -1450,7 +1452,7 @@ checksum = "702fc72eb24e5a1e48ce58027a675bc24edd52096d5397d4aea7c6dd9eca0bd1"
[[package]]
name = "client"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"api",
"arrow-flight",
@@ -1483,7 +1485,7 @@ dependencies = [
"session",
"snafu",
"substrait 0.17.1",
"substrait 0.5.0",
"substrait 0.6.0",
"tokio",
"tokio-stream",
"tonic 0.10.2",
@@ -1513,7 +1515,7 @@ dependencies = [
[[package]]
name = "cmd"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"anymap",
"async-trait",
@@ -1564,7 +1566,7 @@ dependencies = [
"session",
"snafu",
"store-api",
"substrait 0.5.0",
"substrait 0.6.0",
"table",
"temp-env",
"tikv-jemallocator",
@@ -1597,7 +1599,7 @@ checksum = "55b672471b4e9f9e95499ea597ff64941a309b2cdbffcc46f2cc5e2d971fd335"
[[package]]
name = "common-base"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"anymap",
"bitvec",
@@ -1612,7 +1614,7 @@ dependencies = [
[[package]]
name = "common-catalog"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"chrono",
"common-error",
@@ -1623,7 +1625,7 @@ dependencies = [
[[package]]
name = "common-config"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"common-base",
"humantime-serde",
@@ -1636,7 +1638,7 @@ dependencies = [
[[package]]
name = "common-datasource"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"arrow",
"arrow-schema",
@@ -1667,7 +1669,7 @@ dependencies = [
[[package]]
name = "common-decimal"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"arrow",
"bigdecimal",
@@ -1681,7 +1683,7 @@ dependencies = [
[[package]]
name = "common-error"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"snafu",
"strum 0.25.0",
@@ -1689,7 +1691,7 @@ dependencies = [
[[package]]
name = "common-function"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"arc-swap",
"build-data",
@@ -1713,7 +1715,7 @@ dependencies = [
[[package]]
name = "common-greptimedb-telemetry"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"async-trait",
"common-error",
@@ -1732,7 +1734,7 @@ dependencies = [
[[package]]
name = "common-grpc"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"api",
"arrow-flight",
@@ -1762,7 +1764,7 @@ dependencies = [
[[package]]
name = "common-grpc-expr"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"api",
"async-trait",
@@ -1781,7 +1783,7 @@ dependencies = [
[[package]]
name = "common-macro"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"arc-swap",
"common-query",
@@ -1796,7 +1798,7 @@ dependencies = [
[[package]]
name = "common-mem-prof"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"common-error",
"common-macro",
@@ -1809,7 +1811,7 @@ dependencies = [
[[package]]
name = "common-meta"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"api",
"async-recursion",
@@ -1832,6 +1834,7 @@ dependencies = [
"derive_builder 0.12.0",
"etcd-client",
"futures",
"futures-util",
"humantime-serde",
"hyper",
"lazy_static",
@@ -1850,11 +1853,12 @@ dependencies = [
"tokio",
"toml 0.8.8",
"tonic 0.10.2",
"uuid",
]
[[package]]
name = "common-procedure"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"async-stream",
"async-trait",
@@ -1878,7 +1882,7 @@ dependencies = [
[[package]]
name = "common-procedure-test"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"async-trait",
"common-procedure",
@@ -1886,7 +1890,7 @@ dependencies = [
[[package]]
name = "common-query"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"api",
"async-trait",
@@ -1909,7 +1913,7 @@ dependencies = [
[[package]]
name = "common-recordbatch"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"common-error",
"common-macro",
@@ -1926,7 +1930,7 @@ dependencies = [
[[package]]
name = "common-runtime"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"async-trait",
"common-error",
@@ -1946,7 +1950,7 @@ dependencies = [
[[package]]
name = "common-telemetry"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"backtrace",
"common-error",
@@ -1972,8 +1976,11 @@ dependencies = [
[[package]]
name = "common-test-util"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"client",
"common-query",
"common-recordbatch",
"once_cell",
"rand",
"tempfile",
@@ -1981,7 +1988,7 @@ dependencies = [
[[package]]
name = "common-time"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"arrow",
"chrono",
@@ -1997,7 +2004,7 @@ dependencies = [
[[package]]
name = "common-version"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"build-data",
]
@@ -2627,7 +2634,7 @@ dependencies = [
[[package]]
name = "datanode"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"api",
"arrow-flight",
@@ -2687,7 +2694,7 @@ dependencies = [
"snafu",
"sql",
"store-api",
"substrait 0.5.0",
"substrait 0.6.0",
"table",
"tokio",
"tokio-stream",
@@ -2701,7 +2708,7 @@ dependencies = [
[[package]]
name = "datatypes"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"arrow",
"arrow-array",
@@ -3162,7 +3169,7 @@ dependencies = [
[[package]]
name = "file-engine"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"api",
"async-trait",
@@ -3293,7 +3300,7 @@ checksum = "6c2141d6d6c8512188a7891b4b01590a45f6dac67afb4f255c4124dbb86d4eaa"
[[package]]
name = "frontend"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"api",
"arc-swap",
@@ -3357,7 +3364,7 @@ dependencies = [
"sqlparser 0.38.0 (git+https://github.com/GreptimeTeam/sqlparser-rs.git?rev=6a93567ae38d42be5c8d08b13c8ff4dde26502ef)",
"store-api",
"strfmt",
"substrait 0.5.0",
"substrait 0.6.0",
"table",
"tokio",
"toml 0.8.8",
@@ -4011,7 +4018,7 @@ dependencies = [
[[package]]
name = "index"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"async-trait",
"asynchronous-codec",
@@ -4029,7 +4036,7 @@ dependencies = [
"prost 0.12.3",
"rand",
"regex",
"regex-automata 0.1.10",
"regex-automata 0.2.0",
"snafu",
"tokio",
"tokio-util",
@@ -4491,12 +4498,13 @@ checksum = "b5e6163cb8c49088c2c36f57875e58ccd8c87c7427f7fbd50ea6710b2f3f2e8f"
[[package]]
name = "log-store"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"async-stream",
"async-trait",
"byteorder",
"bytes",
"chrono",
"common-base",
"common-config",
"common-error",
@@ -4505,13 +4513,16 @@ dependencies = [
"common-runtime",
"common-telemetry",
"common-test-util",
"common-time",
"dashmap",
"futures",
"futures-util",
"itertools 0.10.5",
"protobuf",
"protobuf-build",
"raft-engine",
"rand",
"rand_distr",
"rskafka",
"serde",
"serde_json",
@@ -4519,6 +4530,7 @@ dependencies = [
"store-api",
"tokio",
"tokio-util",
"uuid",
]
[[package]]
@@ -4765,7 +4777,7 @@ dependencies = [
[[package]]
name = "meta-client"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"api",
"async-trait",
@@ -4795,7 +4807,7 @@ dependencies = [
[[package]]
name = "meta-srv"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"anymap",
"api",
@@ -4824,6 +4836,7 @@ dependencies = [
"futures",
"h2",
"http-body",
"humantime",
"humantime-serde",
"itertools 0.10.5",
"lazy_static",
@@ -4873,7 +4886,7 @@ dependencies = [
[[package]]
name = "metric-engine"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"api",
"aquamarine",
@@ -4944,7 +4957,7 @@ dependencies = [
[[package]]
name = "mito2"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"anymap",
"api",
@@ -4977,6 +4990,7 @@ dependencies = [
"datatypes",
"futures",
"humantime-serde",
"index",
"lazy_static",
"log-store",
"memcomparable",
@@ -4985,8 +4999,10 @@ dependencies = [
"object-store",
"parquet",
"paste",
"pin-project",
"prometheus",
"prost 0.12.3",
"puffin",
"regex",
"serde",
"serde_json",
@@ -5442,7 +5458,7 @@ dependencies = [
[[package]]
name = "object-store"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"anyhow",
"async-trait",
@@ -5498,9 +5514,9 @@ checksum = "0ab1bc2a289d34bd04a330323ac98a1b4bc82c9d9fcb1e66b63caa84da26b575"
[[package]]
name = "opendal"
version = "0.40.0"
version = "0.44.1"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "ddba7299bab261d3ae2f37617fb7f45b19ed872752bb4e22cf93a69d979366c5"
checksum = "bc0ad72f7b44ca4ae59d27ea151fdc6f37305cf6efe099bdaedbb30ec34579c0"
dependencies = [
"anyhow",
"async-compat",
@@ -5511,15 +5527,15 @@ dependencies = [
"chrono",
"flagset",
"futures",
"getrandom",
"http",
"hyper",
"log",
"md-5",
"once_cell",
"parking_lot 0.12.1",
"percent-encoding",
"pin-project",
"quick-xml 0.29.0",
"quick-xml 0.30.0",
"reqsign",
"reqwest",
"serde",
@@ -5687,7 +5703,7 @@ dependencies = [
[[package]]
name = "operator"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"api",
"async-trait",
@@ -5731,7 +5747,7 @@ dependencies = [
"sql",
"sqlparser 0.38.0 (git+https://github.com/GreptimeTeam/sqlparser-rs.git?rev=6a93567ae38d42be5c8d08b13c8ff4dde26502ef)",
"store-api",
"substrait 0.5.0",
"substrait 0.6.0",
"table",
"tokio",
"tonic 0.10.2",
@@ -5962,7 +5978,7 @@ dependencies = [
[[package]]
name = "partition"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"api",
"async-trait",
@@ -6281,7 +6297,7 @@ dependencies = [
[[package]]
name = "plugins"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"auth",
"common-base",
@@ -6539,8 +6555,9 @@ dependencies = [
[[package]]
name = "promql"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"ahash 0.8.6",
"async-recursion",
"async-trait",
"bytemuck",
@@ -6548,6 +6565,7 @@ dependencies = [
"common-catalog",
"common-error",
"common-macro",
"common-query",
"common-recordbatch",
"common-telemetry",
"datafusion",
@@ -6748,7 +6766,7 @@ dependencies = [
[[package]]
name = "puffin"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"async-trait",
"bitflags 2.4.1",
@@ -6859,7 +6877,7 @@ dependencies = [
[[package]]
name = "query"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"ahash 0.8.6",
"api",
@@ -6917,7 +6935,7 @@ dependencies = [
"stats-cli",
"store-api",
"streaming-stats",
"substrait 0.5.0",
"substrait 0.6.0",
"table",
"tokio",
"tokio-stream",
@@ -6934,9 +6952,9 @@ dependencies = [
[[package]]
name = "quick-xml"
version = "0.29.0"
version = "0.30.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "81b9228215d82c7b61490fec1de287136b5de6f5700f6e58ea9ad61a7964ca51"
checksum = "eff6510e86862b57b210fd8cbe8ed3f0d7d600b9c2863cd4549a2e033c66e956"
dependencies = [
"memchr",
"serde",
@@ -7133,8 +7151,18 @@ name = "regex-automata"
version = "0.1.10"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "6c230d73fb8d8c1b9c0b3135c5142a8acee3a0558fb8db5cf1cb65f8d7862132"
dependencies = [
"regex-syntax 0.6.29",
]
[[package]]
name = "regex-automata"
version = "0.2.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "e9368763f5a9b804326f3af749e16f9abf378d227bcdee7634b13d8f17793782"
dependencies = [
"fst",
"memchr",
"regex-syntax 0.6.29",
]
@@ -8177,7 +8205,7 @@ checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49"
[[package]]
name = "script"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"api",
"arc-swap",
@@ -8437,7 +8465,7 @@ dependencies = [
[[package]]
name = "servers"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"aide",
"api",
@@ -8533,7 +8561,7 @@ dependencies = [
[[package]]
name = "session"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"api",
"arc-swap",
@@ -8688,6 +8716,9 @@ name = "smallvec"
version = "1.11.2"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "4dccd0940a2dcdf68d092b8cbab7dc0ad8fa938bf95787e1b916b0e3d0e8e970"
dependencies = [
"serde",
]
[[package]]
name = "smartstring"
@@ -8794,7 +8825,7 @@ dependencies = [
[[package]]
name = "sql"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"api",
"common-base",
@@ -8846,7 +8877,7 @@ dependencies = [
[[package]]
name = "sqlness-runner"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"async-trait",
"clap 4.4.11",
@@ -8858,6 +8889,7 @@ dependencies = [
"common-recordbatch",
"common-time",
"serde",
"serde_json",
"sqlness",
"tinytemplate",
"tokio",
@@ -9052,7 +9084,7 @@ dependencies = [
[[package]]
name = "store-api"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"api",
"aquamarine",
@@ -9192,7 +9224,7 @@ dependencies = [
[[package]]
name = "substrait"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"async-recursion",
"async-trait",
@@ -9340,7 +9372,7 @@ dependencies = [
[[package]]
name = "table"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"anymap",
"async-trait",
@@ -9452,7 +9484,7 @@ checksum = "3369f5ac52d5eb6ab48c6b4ffdc8efbcad6b89c765749064ba298f2c68a16a76"
[[package]]
name = "tests-integration"
version = "0.5.0"
version = "0.6.0"
dependencies = [
"api",
"async-trait",
@@ -9508,7 +9540,7 @@ dependencies = [
"sql",
"sqlx",
"store-api",
"substrait 0.5.0",
"substrait 0.6.0",
"table",
"tempfile",
"time",

View File

@@ -58,7 +58,7 @@ members = [
resolver = "2"
[workspace.package]
version = "0.5.0"
version = "0.6.0"
edition = "2021"
license = "Apache-2.0"
@@ -111,7 +111,7 @@ prost = "0.12"
raft-engine = { git = "https://github.com/tikv/raft-engine.git", rev = "22dfb426cd994602b57725ef080287d3e53db479" }
rand = "0.8"
regex = "1.8"
regex-automata = { version = "0.1", features = ["transducer"] }
regex-automata = { version = "0.2", features = ["transducer"] }
reqwest = { version = "0.11", default-features = false, features = [
"json",
"rustls-tls-native-roots",
@@ -121,7 +121,7 @@ rskafka = "0.5"
rust_decimal = "1.33"
serde = { version = "1.0", features = ["derive"] }
serde_json = "1.0"
smallvec = "1"
smallvec = { version = "1", features = ["serde"] }
snafu = "0.7"
# on branch v0.38.x
sqlparser = { git = "https://github.com/GreptimeTeam/sqlparser-rs.git", rev = "6a93567ae38d42be5c8d08b13c8ff4dde26502ef", features = [
@@ -169,6 +169,7 @@ datanode = { path = "src/datanode" }
datatypes = { path = "src/datatypes" }
file-engine = { path = "src/file-engine" }
frontend = { path = "src/frontend" }
index = { path = "src/index" }
log-store = { path = "src/log-store" }
meta-client = { path = "src/meta-client" }
meta-srv = { path = "src/meta-srv" }
@@ -179,6 +180,7 @@ operator = { path = "src/operator" }
partition = { path = "src/partition" }
plugins = { path = "src/plugins" }
promql = { path = "src/promql" }
puffin = { path = "src/puffin" }
query = { path = "src/query" }
script = { path = "src/script" }
servers = { path = "src/servers" }

View File

@@ -1,8 +1,8 @@
<p align="center">
<picture>
<source media="(prefers-color-scheme: light)" srcset="https://cdn.jsdelivr.net/gh/GreptimeTeam/greptimedb@develop/docs/logo-text-padding.png">
<source media="(prefers-color-scheme: dark)" srcset="https://cdn.jsdelivr.net/gh/GreptimeTeam/greptimedb@develop/docs/logo-text-padding-dark.png">
<img alt="GreptimeDB Logo" src="https://cdn.jsdelivr.net/gh/GreptimeTeam/greptimedb@develop/docs/logo-text-padding.png" width="400px">
<source media="(prefers-color-scheme: light)" srcset="https://cdn.jsdelivr.net/gh/GreptimeTeam/greptimedb@main/docs/logo-text-padding.png">
<source media="(prefers-color-scheme: dark)" srcset="https://cdn.jsdelivr.net/gh/GreptimeTeam/greptimedb@main/docs/logo-text-padding-dark.png">
<img alt="GreptimeDB Logo" src="https://cdn.jsdelivr.net/gh/GreptimeTeam/greptimedb@main/docs/logo-text-padding.png" width="400px">
</picture>
</p>
@@ -12,11 +12,11 @@
</h3>
<p align="center">
<a href="https://codecov.io/gh/GrepTimeTeam/greptimedb"><img src="https://codecov.io/gh/GrepTimeTeam/greptimedb/branch/develop/graph/badge.svg?token=FITFDI3J3C"></img></a>
<a href="https://codecov.io/gh/GrepTimeTeam/greptimedb"><img src="https://codecov.io/gh/GrepTimeTeam/greptimedb/branch/main/graph/badge.svg?token=FITFDI3J3C"></img></a>
&nbsp;
<a href="https://github.com/GreptimeTeam/greptimedb/actions/workflows/develop.yml"><img src="https://github.com/GreptimeTeam/greptimedb/actions/workflows/develop.yml/badge.svg" alt="CI"></img></a>
&nbsp;
<a href="https://github.com/greptimeTeam/greptimedb/blob/develop/LICENSE"><img src="https://img.shields.io/github/license/greptimeTeam/greptimedb"></a>
<a href="https://github.com/greptimeTeam/greptimedb/blob/main/LICENSE"><img src="https://img.shields.io/github/license/greptimeTeam/greptimedb"></a>
</p>
<p align="center">
@@ -29,21 +29,17 @@
## What is GreptimeDB
GreptimeDB is an open-source time-series database with a special focus on
scalability, analytical capabilities and efficiency. It's designed to work on
infrastructure of the cloud era, and users benefit from its elasticity and commodity
storage.
GreptimeDB is an open-source time-series database focusing on efficiency, scalability, and analytical capabilities.
It's designed to work on infrastructure of the cloud era, and users benefit from its elasticity and commodity storage.
Our core developers have been building time-series data platform
for years. Based on their best-practices, GreptimeDB is born to give you:
Our core developers have been building time-series data platforms for years. Based on their best-practices, GreptimeDB is born to give you:
- A standalone binary that scales to highly-available distributed cluster, providing a transparent experience for cluster users
- Optimized columnar layout for handling time-series data; compacted, compressed, and stored on various storage backends
- Flexible indexes, tackling high cardinality issues down
- Distributed, parallel query execution, leveraging elastic computing resource
- Native SQL, and Python scripting for advanced analytical scenarios
- Widely adopted database protocols and APIs, native PromQL supports
- Extensible table engine architecture for extensive workloads
- Optimized columnar layout for handling time-series data; compacted, compressed, and stored on various storage backends, particularly cloud object storage with 50x cost efficiency.
- Fully open-source distributed cluster architecture that harnesses the power of cloud-native elastic computing resources.
- Seamless scalability from a standalone binary at edge to a robust, highly available distributed cluster in cloud, with a transparent experience for both developers and administrators.
- Native SQL and PromQL for queries, and Python scripting to facilitate complex analytical tasks.
- Flexible indexing capabilities and distributed, parallel-processing query engine, tackling high cardinality issues down.
- Widely adopted database protocols and APIs, including MySQL, PostgreSQL, and Prometheus Remote Storage, etc.
## Quick Start
@@ -168,7 +164,7 @@ In addition, you may:
GreptimeDB uses the [Apache 2.0 license][1] to strike a balance between
open contributions and allowing you to use the software however you want.
[1]: <https://github.com/greptimeTeam/greptimedb/blob/develop/LICENSE>
[1]: <https://github.com/greptimeTeam/greptimedb/blob/main/LICENSE>
## Contributing

View File

@@ -34,11 +34,7 @@ connect_timeout = "1s"
tcp_nodelay = true
# WAL options.
# Currently, users are expected to choose the wal through the provider field.
# When a wal provider is chose, the user should comment out all other wal config
# except those corresponding to the chosen one.
[wal]
# WAL data directory
provider = "raft_engine"
# Raft-engine wal options, see `standalone.example.toml`.
@@ -51,9 +47,10 @@ sync_write = false
# Kafka wal options, see `standalone.example.toml`.
# broker_endpoints = ["127.0.0.1:9092"]
# max_batch_size = "4MB"
# Warning: Kafka has a default limit of 1MB per message in a topic.
# max_batch_size = "1MB"
# linger = "200ms"
# produce_record_timeout = "100ms"
# consumer_wait_timeout = "100ms"
# backoff_init = "500ms"
# backoff_max = "10s"
# backoff_base = 2
@@ -116,6 +113,8 @@ sst_write_buffer_size = "8MB"
scan_parallelism = 0
# Capacity of the channel to send data from parallel scan tasks to the main task (default 32).
parallel_scan_channel_size = 32
# Whether to allow stale WAL entries read during replay.
allow_stale_entries = false
# Log options, see `standalone.example.toml`
# [logging]
@@ -129,11 +128,10 @@ parallel_scan_channel_size = 32
# [export_metrics]
# whether enable export metrics, default is false
# enable = false
# The url of metrics export endpoint, default is `frontend` default HTTP endpoint.
# endpoint = "127.0.0.1:4000"
# The database name of exported metrics stores, user needs to specify a valid database
# db = ""
# The interval of export metrics
# write_interval = "30s"
# [export_metrics.remote_write]
# The url the metrics send to. The url is empty by default, url example: `http://127.0.0.1:4000/v1/prometheus/write?db=information_schema`
# url = ""
# HTTP headers of Prometheus remote-write carry
# headers = {}

View File

@@ -87,11 +87,8 @@ tcp_nodelay = true
# [export_metrics]
# whether enable export metrics, default is false
# enable = false
# The url of metrics export endpoint, default is `frontend` default HTTP endpoint.
# endpoint = "127.0.0.1:4000"
# The database name of exported metrics stores, user needs to specify a valid database
# db = ""
# The interval of export metrics
# write_interval = "30s"
# HTTP headers of Prometheus remote-write carry
# headers = {}
# for `frontend`, `self_import` is recommend to collect metrics generated by itself
# [export_metrics.self_import]
# db = "information_schema"

View File

@@ -64,8 +64,6 @@ provider = "raft_engine"
# selector_type = "round_robin"
# A Kafka topic is constructed by concatenating `topic_name_prefix` and `topic_id`.
# topic_name_prefix = "greptimedb_wal_topic"
# Number of partitions per topic.
# num_partitions = 1
# Expected number of replicas of each partition.
# replication_factor = 1
# Above which a topic creation operation will be cancelled.
@@ -86,11 +84,10 @@ provider = "raft_engine"
# [export_metrics]
# whether enable export metrics, default is false
# enable = false
# The url of metrics export endpoint, default is `frontend` default HTTP endpoint.
# endpoint = "127.0.0.1:4000"
# The database name of exported metrics stores, user needs to specify a valid database
# db = ""
# The interval of export metrics
# write_interval = "30s"
# [export_metrics.remote_write]
# The url the metrics send to. The url is empty by default, url example: `http://127.0.0.1:4000/v1/prometheus/write?db=information_schema`
# url = ""
# HTTP headers of Prometheus remote-write carry
# headers = {}

View File

@@ -100,29 +100,29 @@ provider = "raft_engine"
# Available selector types:
# - "round_robin" (default)
# selector_type = "round_robin"
# A Kafka topic is constructed by concatenating `topic_name_prefix` and `topic_id`.
# The prefix of topic name.
# topic_name_prefix = "greptimedb_wal_topic"
# Number of partitions per topic.
# num_partitions = 1
# Expected number of replicas of each partition.
# The number of replicas of each partition.
# Warning: the replication factor must be positive and must not be greater than the number of broker endpoints.
# replication_factor = 1
# The maximum log size a kafka batch producer could buffer.
# max_batch_size = "4MB"
# The linger duration of a kafka batch producer.
# The max size of a single producer batch.
# Warning: Kafka has a default limit of 1MB per message in a topic.
# max_batch_size = "1MB"
# The linger duration.
# linger = "200ms"
# The maximum amount of time (in milliseconds) to wait for Kafka records to be returned.
# produce_record_timeout = "100ms"
# Above which a topic creation operation will be cancelled.
# The consumer wait timeout.
# consumer_wait_timeout = "100ms"
# Create topic timeout.
# create_topic_timeout = "30s"
# The initial backoff for kafka clients.
# The initial backoff delay.
# backoff_init = "500ms"
# The maximum backoff for kafka clients.
# The maximum backoff delay.
# backoff_max = "10s"
# Exponential backoff rate, i.e. next backoff = base * current backoff.
# backoff_base = 2
# Stop reconnecting if the total wait time reaches the deadline. If this config is missing, the reconnecting won't terminate.
# The deadline of retries.
# backoff_deadline = "5mins"
# WAL data directory
@@ -137,6 +137,12 @@ purge_interval = "10m"
read_batch_size = 128
# Whether to sync log file after every write.
sync_write = false
# Whether to reuse logically truncated log files.
enable_log_recycle = true
# Whether to pre-create log files on start up
prefill_log_files = false
# Duration for fsyncing log files.
sync_period = "1000ms"
# Metadata storage options.
[metadata_store]
@@ -207,6 +213,8 @@ sst_write_buffer_size = "8MB"
scan_parallelism = 0
# Capacity of the channel to send data from parallel scan tasks to the main task (default 32).
parallel_scan_channel_size = 32
# Whether to allow stale WAL entries read during replay.
allow_stale_entries = false
# Log options
# [logging]
@@ -230,11 +238,8 @@ parallel_scan_channel_size = 32
# [export_metrics]
# whether enable export metrics, default is false
# enable = false
# The url of metrics export endpoint, default is `frontend` default HTTP endpoint.
# endpoint = "127.0.0.1:4000"
# The database name of exported metrics stores, user needs to specify a valid database
# db = ""
# The interval of export metrics
# write_interval = "30s"
# HTTP headers of Prometheus remote-write carry
# headers = {}
# for `standalone`, `self_import` is recommend to collect metrics generated by itself
# [export_metrics.self_import]
# db = "information_schema"

View File

@@ -11,6 +11,7 @@ testing = []
api.workspace = true
arc-swap = "1.0"
arrow-schema.workspace = true
arrow.workspace = true
async-stream.workspace = true
async-trait = "0.1"
build-data = "0.1"
@@ -29,6 +30,7 @@ datafusion.workspace = true
datatypes.workspace = true
futures = "0.3"
futures-util.workspace = true
itertools.workspace = true
lazy_static.workspace = true
meta-client.workspace = true
moka = { workspace = true, features = ["future"] }

View File

@@ -13,20 +13,25 @@
// limitations under the License.
mod columns;
mod key_column_usage;
mod memory_table;
mod predicate;
mod runtime_metrics;
mod schemata;
mod table_names;
mod tables;
use std::collections::HashMap;
use std::sync::{Arc, Weak};
use common_catalog::consts::{self, INFORMATION_SCHEMA_NAME};
use common_catalog::consts::{self, DEFAULT_CATALOG_NAME, INFORMATION_SCHEMA_NAME};
use common_error::ext::BoxedError;
use common_recordbatch::{RecordBatchStreamWrapper, SendableRecordBatchStream};
use datatypes::schema::SchemaRef;
use futures_util::StreamExt;
use lazy_static::lazy_static;
use paste::paste;
pub(crate) use predicate::Predicates;
use snafu::ResultExt;
use store_api::data_source::DataSource;
use store_api::storage::{ScanRequest, TableId};
@@ -40,7 +45,10 @@ pub use table_names::*;
use self::columns::InformationSchemaColumns;
use crate::error::Result;
use crate::information_schema::key_column_usage::InformationSchemaKeyColumnUsage;
use crate::information_schema::memory_table::{get_schema_columns, MemoryTable};
use crate::information_schema::runtime_metrics::InformationSchemaMetrics;
use crate::information_schema::schemata::InformationSchemaSchemata;
use crate::information_schema::tables::InformationSchemaTables;
use crate::CatalogManager;
@@ -50,7 +58,22 @@ lazy_static! {
ENGINES,
COLUMN_PRIVILEGES,
COLUMN_STATISTICS,
BUILD_INFO,
CHARACTER_SETS,
COLLATIONS,
COLLATION_CHARACTER_SET_APPLICABILITY,
CHECK_CONSTRAINTS,
EVENTS,
FILES,
OPTIMIZER_TRACE,
PARAMETERS,
PROFILING,
REFERENTIAL_CONSTRAINTS,
ROUTINES,
SCHEMA_PRIVILEGES,
TABLE_PRIVILEGES,
TRIGGERS,
GLOBAL_STATUS,
SESSION_STATUS,
];
}
@@ -120,12 +143,32 @@ impl InformationSchemaProvider {
fn build_tables(&mut self) {
let mut tables = HashMap::new();
// Carefully consider the tables that may expose sensitive cluster configurations,
// authentication details, and other critical information.
// Only put these tables under `greptime` catalog to prevent info leak.
if self.catalog_name == DEFAULT_CATALOG_NAME {
tables.insert(
RUNTIME_METRICS.to_string(),
self.build_table(RUNTIME_METRICS).unwrap(),
);
tables.insert(
BUILD_INFO.to_string(),
self.build_table(BUILD_INFO).unwrap(),
);
}
tables.insert(TABLES.to_string(), self.build_table(TABLES).unwrap());
tables.insert(SCHEMATA.to_string(), self.build_table(SCHEMATA).unwrap());
tables.insert(COLUMNS.to_string(), self.build_table(COLUMNS).unwrap());
tables.insert(
KEY_COLUMN_USAGE.to_string(),
self.build_table(KEY_COLUMN_USAGE).unwrap(),
);
// Add memory tables
for name in MEMORY_TABLES.iter() {
tables.insert((*name).to_string(), self.build_table(name).unwrap());
tables.insert((*name).to_string(), self.build_table(name).expect(name));
}
self.tables = tables;
@@ -134,7 +177,7 @@ impl InformationSchemaProvider {
fn build_table(&self, name: &str) -> Option<TableRef> {
self.information_table(name).map(|table| {
let table_info = Self::table_info(self.catalog_name.clone(), &table);
let filter_pushdown = FilterPushDownType::Unsupported;
let filter_pushdown = FilterPushDownType::Inexact;
let thin_table = ThinTable::new(table_info, filter_pushdown);
let data_source = Arc::new(InformationTableDataSource::new(table));
@@ -156,6 +199,33 @@ impl InformationSchemaProvider {
COLUMN_PRIVILEGES => setup_memory_table!(COLUMN_PRIVILEGES),
COLUMN_STATISTICS => setup_memory_table!(COLUMN_STATISTICS),
BUILD_INFO => setup_memory_table!(BUILD_INFO),
CHARACTER_SETS => setup_memory_table!(CHARACTER_SETS),
COLLATIONS => setup_memory_table!(COLLATIONS),
COLLATION_CHARACTER_SET_APPLICABILITY => {
setup_memory_table!(COLLATION_CHARACTER_SET_APPLICABILITY)
}
CHECK_CONSTRAINTS => setup_memory_table!(CHECK_CONSTRAINTS),
EVENTS => setup_memory_table!(EVENTS),
FILES => setup_memory_table!(FILES),
OPTIMIZER_TRACE => setup_memory_table!(OPTIMIZER_TRACE),
PARAMETERS => setup_memory_table!(PARAMETERS),
PROFILING => setup_memory_table!(PROFILING),
REFERENTIAL_CONSTRAINTS => setup_memory_table!(REFERENTIAL_CONSTRAINTS),
ROUTINES => setup_memory_table!(ROUTINES),
SCHEMA_PRIVILEGES => setup_memory_table!(SCHEMA_PRIVILEGES),
TABLE_PRIVILEGES => setup_memory_table!(TABLE_PRIVILEGES),
TRIGGERS => setup_memory_table!(TRIGGERS),
GLOBAL_STATUS => setup_memory_table!(GLOBAL_STATUS),
SESSION_STATUS => setup_memory_table!(SESSION_STATUS),
KEY_COLUMN_USAGE => Some(Arc::new(InformationSchemaKeyColumnUsage::new(
self.catalog_name.clone(),
self.catalog_manager.clone(),
)) as _),
SCHEMATA => Some(Arc::new(InformationSchemaSchemata::new(
self.catalog_name.clone(),
self.catalog_manager.clone(),
)) as _),
RUNTIME_METRICS => Some(Arc::new(InformationSchemaMetrics::new())),
_ => None,
}
}
@@ -187,7 +257,7 @@ trait InformationTable {
fn schema(&self) -> SchemaRef;
fn to_stream(&self) -> Result<SendableRecordBatchStream>;
fn to_stream(&self, request: ScanRequest) -> Result<SendableRecordBatchStream>;
fn table_type(&self) -> TableType {
TableType::Temporary
@@ -221,7 +291,7 @@ impl DataSource for InformationTableDataSource {
&self,
request: ScanRequest,
) -> std::result::Result<SendableRecordBatchStream, BoxedError> {
let projection = request.projection;
let projection = request.projection.clone();
let projected_schema = match &projection {
Some(projection) => self.try_project(projection)?,
None => self.table.schema(),
@@ -229,7 +299,7 @@ impl DataSource for InformationTableDataSource {
let stream = self
.table
.to_stream()
.to_stream(request)
.map_err(BoxedError::new)
.context(TablesRecordBatchSnafu)
.map_err(BoxedError::new)?

View File

@@ -29,14 +29,16 @@ use datafusion::physical_plan::SendableRecordBatchStream as DfSendableRecordBatc
use datatypes::prelude::{ConcreteDataType, DataType};
use datatypes::scalars::ScalarVectorBuilder;
use datatypes::schema::{ColumnSchema, Schema, SchemaRef};
use datatypes::value::Value;
use datatypes::vectors::{StringVectorBuilder, VectorRef};
use snafu::{OptionExt, ResultExt};
use store_api::storage::TableId;
use store_api::storage::{ScanRequest, TableId};
use super::{InformationTable, COLUMNS};
use crate::error::{
CreateRecordBatchSnafu, InternalSnafu, Result, UpgradeWeakCatalogManagerRefSnafu,
};
use crate::information_schema::Predicates;
use crate::CatalogManager;
pub(super) struct InformationSchemaColumns {
@@ -51,6 +53,10 @@ const TABLE_NAME: &str = "table_name";
const COLUMN_NAME: &str = "column_name";
const DATA_TYPE: &str = "data_type";
const SEMANTIC_TYPE: &str = "semantic_type";
const COLUMN_DEFAULT: &str = "column_default";
const IS_NULLABLE: &str = "is_nullable";
const COLUMN_TYPE: &str = "column_type";
const COLUMN_COMMENT: &str = "column_comment";
impl InformationSchemaColumns {
pub(super) fn new(catalog_name: String, catalog_manager: Weak<dyn CatalogManager>) -> Self {
@@ -69,6 +75,10 @@ impl InformationSchemaColumns {
ColumnSchema::new(COLUMN_NAME, ConcreteDataType::string_datatype(), false),
ColumnSchema::new(DATA_TYPE, ConcreteDataType::string_datatype(), false),
ColumnSchema::new(SEMANTIC_TYPE, ConcreteDataType::string_datatype(), false),
ColumnSchema::new(COLUMN_DEFAULT, ConcreteDataType::string_datatype(), true),
ColumnSchema::new(IS_NULLABLE, ConcreteDataType::string_datatype(), false),
ColumnSchema::new(COLUMN_TYPE, ConcreteDataType::string_datatype(), false),
ColumnSchema::new(COLUMN_COMMENT, ConcreteDataType::string_datatype(), true),
]))
}
@@ -94,14 +104,14 @@ impl InformationTable for InformationSchemaColumns {
self.schema.clone()
}
fn to_stream(&self) -> Result<SendableRecordBatchStream> {
fn to_stream(&self, request: ScanRequest) -> Result<SendableRecordBatchStream> {
let schema = self.schema.arrow_schema().clone();
let mut builder = self.builder();
let stream = Box::pin(DfRecordBatchStreamAdapter::new(
schema,
futures::stream::once(async move {
builder
.make_columns()
.make_columns(Some(request))
.await
.map(|x| x.into_df_record_batch())
.map_err(Into::into)
@@ -126,6 +136,11 @@ struct InformationSchemaColumnsBuilder {
column_names: StringVectorBuilder,
data_types: StringVectorBuilder,
semantic_types: StringVectorBuilder,
column_defaults: StringVectorBuilder,
is_nullables: StringVectorBuilder,
column_types: StringVectorBuilder,
column_comments: StringVectorBuilder,
}
impl InformationSchemaColumnsBuilder {
@@ -144,16 +159,21 @@ impl InformationSchemaColumnsBuilder {
column_names: StringVectorBuilder::with_capacity(42),
data_types: StringVectorBuilder::with_capacity(42),
semantic_types: StringVectorBuilder::with_capacity(42),
column_defaults: StringVectorBuilder::with_capacity(42),
is_nullables: StringVectorBuilder::with_capacity(42),
column_types: StringVectorBuilder::with_capacity(42),
column_comments: StringVectorBuilder::with_capacity(42),
}
}
/// Construct the `information_schema.columns` virtual table
async fn make_columns(&mut self) -> Result<RecordBatch> {
async fn make_columns(&mut self, request: Option<ScanRequest>) -> Result<RecordBatch> {
let catalog_name = self.catalog_name.clone();
let catalog_manager = self
.catalog_manager
.upgrade()
.context(UpgradeWeakCatalogManagerRefSnafu)?;
let predicates = Predicates::from_scan_request(&request);
for schema_name in catalog_manager.schema_names(&catalog_name).await? {
if !catalog_manager
@@ -184,12 +204,12 @@ impl InformationSchemaColumnsBuilder {
};
self.add_column(
&predicates,
&catalog_name,
&schema_name,
&table_name,
&column.name,
&column.data_type.name(),
semantic_type,
column,
);
}
} else {
@@ -203,19 +223,48 @@ impl InformationSchemaColumnsBuilder {
fn add_column(
&mut self,
predicates: &Predicates,
catalog_name: &str,
schema_name: &str,
table_name: &str,
column_name: &str,
data_type: &str,
semantic_type: &str,
column_schema: &ColumnSchema,
) {
let data_type = &column_schema.data_type.name();
let row = [
(TABLE_CATALOG, &Value::from(catalog_name)),
(TABLE_SCHEMA, &Value::from(schema_name)),
(TABLE_NAME, &Value::from(table_name)),
(COLUMN_NAME, &Value::from(column_schema.name.as_str())),
(DATA_TYPE, &Value::from(data_type.as_str())),
(SEMANTIC_TYPE, &Value::from(semantic_type)),
];
if !predicates.eval(&row) {
return;
}
self.catalog_names.push(Some(catalog_name));
self.schema_names.push(Some(schema_name));
self.table_names.push(Some(table_name));
self.column_names.push(Some(column_name));
self.column_names.push(Some(&column_schema.name));
self.data_types.push(Some(data_type));
self.semantic_types.push(Some(semantic_type));
self.column_defaults.push(
column_schema
.default_constraint()
.map(|s| format!("{}", s))
.as_deref(),
);
if column_schema.is_nullable() {
self.is_nullables.push(Some("Yes"));
} else {
self.is_nullables.push(Some("No"));
}
self.column_types.push(Some(data_type));
self.column_comments
.push(column_schema.column_comment().map(|x| x.as_ref()));
}
fn finish(&mut self) -> Result<RecordBatch> {
@@ -226,6 +275,10 @@ impl InformationSchemaColumnsBuilder {
Arc::new(self.column_names.finish()),
Arc::new(self.data_types.finish()),
Arc::new(self.semantic_types.finish()),
Arc::new(self.column_defaults.finish()),
Arc::new(self.is_nullables.finish()),
Arc::new(self.column_types.finish()),
Arc::new(self.column_comments.finish()),
];
RecordBatch::new(self.schema.clone(), columns).context(CreateRecordBatchSnafu)
@@ -244,7 +297,7 @@ impl DfPartitionStream for InformationSchemaColumns {
schema,
futures::stream::once(async move {
builder
.make_columns()
.make_columns(None)
.await
.map(|x| x.into_df_record_batch())
.map_err(Into::into)

View File

@@ -0,0 +1,347 @@
// 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, Weak};
use arrow_schema::SchemaRef as ArrowSchemaRef;
use common_catalog::consts::INFORMATION_SCHEMA_KEY_COLUMN_USAGE_TABLE_ID;
use common_error::ext::BoxedError;
use common_query::physical_plan::TaskContext;
use common_recordbatch::adapter::RecordBatchStreamAdapter;
use common_recordbatch::{RecordBatch, SendableRecordBatchStream};
use datafusion::physical_plan::stream::RecordBatchStreamAdapter as DfRecordBatchStreamAdapter;
use datafusion::physical_plan::streaming::PartitionStream as DfPartitionStream;
use datafusion::physical_plan::SendableRecordBatchStream as DfSendableRecordBatchStream;
use datatypes::prelude::{ConcreteDataType, ScalarVectorBuilder, VectorRef};
use datatypes::schema::{ColumnSchema, Schema, SchemaRef};
use datatypes::value::Value;
use datatypes::vectors::{StringVectorBuilder, UInt32VectorBuilder};
use snafu::{OptionExt, ResultExt};
use store_api::storage::{ScanRequest, TableId};
use super::KEY_COLUMN_USAGE;
use crate::error::{
CreateRecordBatchSnafu, InternalSnafu, Result, UpgradeWeakCatalogManagerRefSnafu,
};
use crate::information_schema::{InformationTable, Predicates};
use crate::CatalogManager;
const CONSTRAINT_SCHEMA: &str = "constraint_schema";
const CONSTRAINT_NAME: &str = "constraint_name";
const TABLE_CATALOG: &str = "table_catalog";
const TABLE_SCHEMA: &str = "table_schema";
const TABLE_NAME: &str = "table_name";
const COLUMN_NAME: &str = "column_name";
const ORDINAL_POSITION: &str = "ordinal_position";
/// The virtual table implementation for `information_schema.KEY_COLUMN_USAGE`.
pub(super) struct InformationSchemaKeyColumnUsage {
schema: SchemaRef,
catalog_name: String,
catalog_manager: Weak<dyn CatalogManager>,
}
impl InformationSchemaKeyColumnUsage {
pub(super) fn new(catalog_name: String, catalog_manager: Weak<dyn CatalogManager>) -> Self {
Self {
schema: Self::schema(),
catalog_name,
catalog_manager,
}
}
pub(crate) fn schema() -> SchemaRef {
Arc::new(Schema::new(vec![
ColumnSchema::new(
"constraint_catalog",
ConcreteDataType::string_datatype(),
false,
),
ColumnSchema::new(
CONSTRAINT_SCHEMA,
ConcreteDataType::string_datatype(),
false,
),
ColumnSchema::new(CONSTRAINT_NAME, ConcreteDataType::string_datatype(), false),
ColumnSchema::new(TABLE_CATALOG, ConcreteDataType::string_datatype(), false),
ColumnSchema::new(TABLE_SCHEMA, ConcreteDataType::string_datatype(), false),
ColumnSchema::new(TABLE_NAME, ConcreteDataType::string_datatype(), false),
ColumnSchema::new(COLUMN_NAME, ConcreteDataType::string_datatype(), false),
ColumnSchema::new(ORDINAL_POSITION, ConcreteDataType::uint32_datatype(), false),
ColumnSchema::new(
"position_in_unique_constraint",
ConcreteDataType::uint32_datatype(),
true,
),
ColumnSchema::new(
"referenced_table_schema",
ConcreteDataType::string_datatype(),
true,
),
ColumnSchema::new(
"referenced_table_name",
ConcreteDataType::string_datatype(),
true,
),
ColumnSchema::new(
"referenced_column_name",
ConcreteDataType::string_datatype(),
true,
),
]))
}
fn builder(&self) -> InformationSchemaKeyColumnUsageBuilder {
InformationSchemaKeyColumnUsageBuilder::new(
self.schema.clone(),
self.catalog_name.clone(),
self.catalog_manager.clone(),
)
}
}
impl InformationTable for InformationSchemaKeyColumnUsage {
fn table_id(&self) -> TableId {
INFORMATION_SCHEMA_KEY_COLUMN_USAGE_TABLE_ID
}
fn table_name(&self) -> &'static str {
KEY_COLUMN_USAGE
}
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
fn to_stream(&self, request: ScanRequest) -> Result<SendableRecordBatchStream> {
let schema = self.schema.arrow_schema().clone();
let mut builder = self.builder();
let stream = Box::pin(DfRecordBatchStreamAdapter::new(
schema,
futures::stream::once(async move {
builder
.make_key_column_usage(Some(request))
.await
.map(|x| x.into_df_record_batch())
.map_err(Into::into)
}),
));
Ok(Box::pin(
RecordBatchStreamAdapter::try_new(stream)
.map_err(BoxedError::new)
.context(InternalSnafu)?,
))
}
}
/// Builds the `information_schema.KEY_COLUMN_USAGE` table row by row
///
/// Columns are based on <https://dev.mysql.com/doc/refman/8.2/en/information-schema-key-column-usage-table.html>
struct InformationSchemaKeyColumnUsageBuilder {
schema: SchemaRef,
catalog_name: String,
catalog_manager: Weak<dyn CatalogManager>,
constraint_catalog: StringVectorBuilder,
constraint_schema: StringVectorBuilder,
constraint_name: StringVectorBuilder,
table_catalog: StringVectorBuilder,
table_schema: StringVectorBuilder,
table_name: StringVectorBuilder,
column_name: StringVectorBuilder,
ordinal_position: UInt32VectorBuilder,
position_in_unique_constraint: UInt32VectorBuilder,
referenced_table_schema: StringVectorBuilder,
referenced_table_name: StringVectorBuilder,
referenced_column_name: StringVectorBuilder,
}
impl InformationSchemaKeyColumnUsageBuilder {
fn new(
schema: SchemaRef,
catalog_name: String,
catalog_manager: Weak<dyn CatalogManager>,
) -> Self {
Self {
schema,
catalog_name,
catalog_manager,
constraint_catalog: StringVectorBuilder::with_capacity(42),
constraint_schema: StringVectorBuilder::with_capacity(42),
constraint_name: StringVectorBuilder::with_capacity(42),
table_catalog: StringVectorBuilder::with_capacity(42),
table_schema: StringVectorBuilder::with_capacity(42),
table_name: StringVectorBuilder::with_capacity(42),
column_name: StringVectorBuilder::with_capacity(42),
ordinal_position: UInt32VectorBuilder::with_capacity(42),
position_in_unique_constraint: UInt32VectorBuilder::with_capacity(42),
referenced_table_schema: StringVectorBuilder::with_capacity(42),
referenced_table_name: StringVectorBuilder::with_capacity(42),
referenced_column_name: StringVectorBuilder::with_capacity(42),
}
}
/// Construct the `information_schema.KEY_COLUMN_USAGE` virtual table
async fn make_key_column_usage(&mut self, request: Option<ScanRequest>) -> Result<RecordBatch> {
let catalog_name = self.catalog_name.clone();
let catalog_manager = self
.catalog_manager
.upgrade()
.context(UpgradeWeakCatalogManagerRefSnafu)?;
let predicates = Predicates::from_scan_request(&request);
let mut primary_constraints = vec![];
for schema_name in catalog_manager.schema_names(&catalog_name).await? {
if !catalog_manager
.schema_exists(&catalog_name, &schema_name)
.await?
{
continue;
}
for table_name in catalog_manager
.table_names(&catalog_name, &schema_name)
.await?
{
if let Some(table) = catalog_manager
.table(&catalog_name, &schema_name, &table_name)
.await?
{
let keys = &table.table_info().meta.primary_key_indices;
let schema = table.schema();
for (idx, column) in schema.column_schemas().iter().enumerate() {
if column.is_time_index() {
self.add_key_column_usage(
&predicates,
&schema_name,
"TIME INDEX",
&schema_name,
&table_name,
&column.name,
1, //always 1 for time index
);
}
if keys.contains(&idx) {
primary_constraints.push((
schema_name.clone(),
table_name.clone(),
column.name.clone(),
));
}
// TODO(dimbtp): foreign key constraint not supported yet
}
} else {
unreachable!();
}
}
}
for (i, (schema_name, table_name, column_name)) in
primary_constraints.into_iter().enumerate()
{
self.add_key_column_usage(
&predicates,
&schema_name,
"PRIMARY",
&schema_name,
&table_name,
&column_name,
i as u32 + 1,
);
}
self.finish()
}
// TODO(dimbtp): Foreign key constraint has not `None` value for last 4
// fields, but it is not supported yet.
#[allow(clippy::too_many_arguments)]
fn add_key_column_usage(
&mut self,
predicates: &Predicates,
constraint_schema: &str,
constraint_name: &str,
table_schema: &str,
table_name: &str,
column_name: &str,
ordinal_position: u32,
) {
let row = [
(CONSTRAINT_SCHEMA, &Value::from(constraint_schema)),
(CONSTRAINT_NAME, &Value::from(constraint_name)),
(TABLE_SCHEMA, &Value::from(table_schema)),
(TABLE_NAME, &Value::from(table_name)),
(COLUMN_NAME, &Value::from(column_name)),
(ORDINAL_POSITION, &Value::from(ordinal_position)),
];
if !predicates.eval(&row) {
return;
}
self.constraint_catalog.push(Some("def"));
self.constraint_schema.push(Some(constraint_schema));
self.constraint_name.push(Some(constraint_name));
self.table_catalog.push(Some("def"));
self.table_schema.push(Some(table_schema));
self.table_name.push(Some(table_name));
self.column_name.push(Some(column_name));
self.ordinal_position.push(Some(ordinal_position));
self.position_in_unique_constraint.push(None);
self.referenced_table_schema.push(None);
self.referenced_table_name.push(None);
self.referenced_column_name.push(None);
}
fn finish(&mut self) -> Result<RecordBatch> {
let columns: Vec<VectorRef> = vec![
Arc::new(self.constraint_catalog.finish()),
Arc::new(self.constraint_schema.finish()),
Arc::new(self.constraint_name.finish()),
Arc::new(self.table_catalog.finish()),
Arc::new(self.table_schema.finish()),
Arc::new(self.table_name.finish()),
Arc::new(self.column_name.finish()),
Arc::new(self.ordinal_position.finish()),
Arc::new(self.position_in_unique_constraint.finish()),
Arc::new(self.referenced_table_schema.finish()),
Arc::new(self.referenced_table_name.finish()),
Arc::new(self.referenced_column_name.finish()),
];
RecordBatch::new(self.schema.clone(), columns).context(CreateRecordBatchSnafu)
}
}
impl DfPartitionStream for InformationSchemaKeyColumnUsage {
fn schema(&self) -> &ArrowSchemaRef {
self.schema.arrow_schema()
}
fn execute(&self, _: Arc<TaskContext>) -> DfSendableRecordBatchStream {
let schema = self.schema.arrow_schema().clone();
let mut builder = self.builder();
Box::pin(DfRecordBatchStreamAdapter::new(
schema,
futures::stream::once(async move {
builder
.make_key_column_usage(None)
.await
.map(|x| x.into_df_record_batch())
.map_err(Into::into)
}),
))
}
}

View File

@@ -26,7 +26,7 @@ use datafusion::physical_plan::SendableRecordBatchStream as DfSendableRecordBatc
use datatypes::schema::SchemaRef;
use datatypes::vectors::VectorRef;
use snafu::ResultExt;
use store_api::storage::TableId;
use store_api::storage::{ScanRequest, TableId};
pub use tables::get_schema_columns;
use crate::error::{CreateRecordBatchSnafu, InternalSnafu, Result};
@@ -74,7 +74,7 @@ impl InformationTable for MemoryTable {
self.schema.clone()
}
fn to_stream(&self) -> Result<SendableRecordBatchStream> {
fn to_stream(&self, _request: ScanRequest) -> Result<SendableRecordBatchStream> {
let schema = self.schema.arrow_schema().clone();
let mut builder = self.builder();
let stream = Box::pin(DfRecordBatchStreamAdapter::new(
@@ -169,7 +169,7 @@ mod tests {
assert_eq!("test", table.table_name());
assert_eq!(schema, InformationTable::schema(&table));
let stream = table.to_stream().unwrap();
let stream = table.to_stream(ScanRequest::default()).unwrap();
let batches = RecordBatches::try_collect(stream).await.unwrap();
@@ -198,7 +198,7 @@ mod tests {
assert_eq!("test", table.table_name());
assert_eq!(schema, InformationTable::schema(&table));
let stream = table.to_stream().unwrap();
let stream = table.to_stream(ScanRequest::default()).unwrap();
let batches = RecordBatches::try_collect(stream).await.unwrap();

View File

@@ -17,7 +17,7 @@ use std::sync::Arc;
use common_catalog::consts::MITO_ENGINE;
use datatypes::prelude::{ConcreteDataType, VectorRef};
use datatypes::schema::{ColumnSchema, Schema, SchemaRef};
use datatypes::vectors::StringVector;
use datatypes::vectors::{Int64Vector, StringVector};
use crate::information_schema::table_names::*;
@@ -97,6 +97,320 @@ pub fn get_schema_columns(table_name: &str) -> (SchemaRef, Vec<VectorRef>) {
],
),
CHARACTER_SETS => (
vec![
string_column("CHARACTER_SET_NAME"),
string_column("DEFAULT_COLLATE_NAME"),
string_column("DESCRIPTION"),
bigint_column("MAXLEN"),
],
vec![
Arc::new(StringVector::from(vec!["utf8"])),
Arc::new(StringVector::from(vec!["utf8_bin"])),
Arc::new(StringVector::from(vec!["UTF-8 Unicode"])),
Arc::new(Int64Vector::from_slice([4])),
],
),
COLLATIONS => (
vec![
string_column("COLLATION_NAME"),
string_column("CHARACTER_SET_NAME"),
bigint_column("ID"),
string_column("IS_DEFAULT"),
string_column("IS_COMPILED"),
bigint_column("SORTLEN"),
],
vec![
Arc::new(StringVector::from(vec!["utf8_bin"])),
Arc::new(StringVector::from(vec!["utf8"])),
Arc::new(Int64Vector::from_slice([1])),
Arc::new(StringVector::from(vec!["Yes"])),
Arc::new(StringVector::from(vec!["Yes"])),
Arc::new(Int64Vector::from_slice([1])),
],
),
COLLATION_CHARACTER_SET_APPLICABILITY => (
vec![
string_column("COLLATION_NAME"),
string_column("CHARACTER_SET_NAME"),
],
vec![
Arc::new(StringVector::from(vec!["utf8_bin"])),
Arc::new(StringVector::from(vec!["utf8"])),
],
),
CHECK_CONSTRAINTS => (
string_columns(&[
"CONSTRAINT_CATALOG",
"CONSTRAINT_SCHEMA",
"CONSTRAINT_NAME",
"CHECK_CLAUSE",
]),
// Not support check constraints yet
vec![],
),
EVENTS => (
vec![
string_column("EVENT_CATALOG"),
string_column("EVENT_SCHEMA"),
string_column("EVENT_NAME"),
string_column("DEFINER"),
string_column("TIME_ZONE"),
string_column("EVENT_BODY"),
string_column("EVENT_DEFINITION"),
string_column("EVENT_TYPE"),
datetime_column("EXECUTE_AT"),
bigint_column("INTERVAL_VALUE"),
string_column("INTERVAL_FIELD"),
string_column("SQL_MODE"),
datetime_column("STARTS"),
datetime_column("ENDS"),
string_column("STATUS"),
string_column("ON_COMPLETION"),
datetime_column("CREATED"),
datetime_column("LAST_ALTERED"),
datetime_column("LAST_EXECUTED"),
string_column("EVENT_COMMENT"),
bigint_column("ORIGINATOR"),
string_column("CHARACTER_SET_CLIENT"),
string_column("COLLATION_CONNECTION"),
string_column("DATABASE_COLLATION"),
],
vec![],
),
FILES => (
vec![
bigint_column("FILE_ID"),
string_column("FILE_NAME"),
string_column("FILE_TYPE"),
string_column("TABLESPACE_NAME"),
string_column("TABLE_CATALOG"),
string_column("TABLE_SCHEMA"),
string_column("TABLE_NAME"),
string_column("LOGFILE_GROUP_NAME"),
bigint_column("LOGFILE_GROUP_NUMBER"),
string_column("ENGINE"),
string_column("FULLTEXT_KEYS"),
bigint_column("DELETED_ROWS"),
bigint_column("UPDATE_COUNT"),
bigint_column("FREE_EXTENTS"),
bigint_column("TOTAL_EXTENTS"),
bigint_column("EXTENT_SIZE"),
bigint_column("INITIAL_SIZE"),
bigint_column("MAXIMUM_SIZE"),
bigint_column("AUTOEXTEND_SIZE"),
datetime_column("CREATION_TIME"),
datetime_column("LAST_UPDATE_TIME"),
datetime_column("LAST_ACCESS_TIME"),
datetime_column("RECOVER_TIME"),
bigint_column("TRANSACTION_COUNTER"),
string_column("VERSION"),
string_column("ROW_FORMAT"),
bigint_column("TABLE_ROWS"),
bigint_column("AVG_ROW_LENGTH"),
bigint_column("DATA_LENGTH"),
bigint_column("MAX_DATA_LENGTH"),
bigint_column("INDEX_LENGTH"),
bigint_column("DATA_FREE"),
datetime_column("CREATE_TIME"),
datetime_column("UPDATE_TIME"),
datetime_column("CHECK_TIME"),
string_column("CHECKSUM"),
string_column("STATUS"),
string_column("EXTRA"),
],
vec![],
),
OPTIMIZER_TRACE => (
vec![
string_column("QUERY"),
string_column("TRACE"),
bigint_column("MISSING_BYTES_BEYOND_MAX_MEM_SIZE"),
bigint_column("INSUFFICIENT_PRIVILEGES"),
],
vec![],
),
// MySQL(https://dev.mysql.com/doc/refman/8.2/en/information-schema-parameters-table.html)
// has the spec that is different from
// PostgreSQL(https://www.postgresql.org/docs/current/infoschema-parameters.html).
// Follow `MySQL` spec here.
PARAMETERS => (
vec![
string_column("SPECIFIC_CATALOG"),
string_column("SPECIFIC_SCHEMA"),
string_column("SPECIFIC_NAME"),
bigint_column("ORDINAL_POSITION"),
string_column("PARAMETER_MODE"),
string_column("PARAMETER_NAME"),
string_column("DATA_TYPE"),
bigint_column("CHARACTER_MAXIMUM_LENGTH"),
bigint_column("CHARACTER_OCTET_LENGTH"),
bigint_column("NUMERIC_PRECISION"),
bigint_column("NUMERIC_SCALE"),
bigint_column("DATETIME_PRECISION"),
string_column("CHARACTER_SET_NAME"),
string_column("COLLATION_NAME"),
string_column("DTD_IDENTIFIER"),
string_column("ROUTINE_TYPE"),
],
vec![],
),
PROFILING => (
vec![
bigint_column("QUERY_ID"),
bigint_column("SEQ"),
string_column("STATE"),
bigint_column("DURATION"),
bigint_column("CPU_USER"),
bigint_column("CPU_SYSTEM"),
bigint_column("CONTEXT_VOLUNTARY"),
bigint_column("CONTEXT_INVOLUNTARY"),
bigint_column("BLOCK_OPS_IN"),
bigint_column("BLOCK_OPS_OUT"),
bigint_column("MESSAGES_SENT"),
bigint_column("MESSAGES_RECEIVED"),
bigint_column("PAGE_FAULTS_MAJOR"),
bigint_column("PAGE_FAULTS_MINOR"),
bigint_column("SWAPS"),
string_column("SOURCE_FUNCTION"),
string_column("SOURCE_FILE"),
bigint_column("SOURCE_LINE"),
],
vec![],
),
// TODO: _Must_ reimplement this table when foreign key constraint is supported.
REFERENTIAL_CONSTRAINTS => (
vec![
string_column("CONSTRAINT_CATALOG"),
string_column("CONSTRAINT_SCHEMA"),
string_column("CONSTRAINT_NAME"),
string_column("UNIQUE_CONSTRAINT_CATALOG"),
string_column("UNIQUE_CONSTRAINT_SCHEMA"),
string_column("UNIQUE_CONSTRAINT_NAME"),
string_column("MATCH_OPTION"),
string_column("UPDATE_RULE"),
string_column("DELETE_RULE"),
string_column("TABLE_NAME"),
string_column("REFERENCED_TABLE_NAME"),
],
vec![],
),
ROUTINES => (
vec![
string_column("SPECIFIC_NAME"),
string_column("ROUTINE_CATALOG"),
string_column("ROUTINE_SCHEMA"),
string_column("ROUTINE_NAME"),
string_column("ROUTINE_TYPE"),
string_column("DATA_TYPE"),
bigint_column("CHARACTER_MAXIMUM_LENGTH"),
bigint_column("CHARACTER_OCTET_LENGTH"),
bigint_column("NUMERIC_PRECISION"),
bigint_column("NUMERIC_SCALE"),
bigint_column("DATETIME_PRECISION"),
string_column("CHARACTER_SET_NAME"),
string_column("COLLATION_NAME"),
string_column("DTD_IDENTIFIER"),
string_column("ROUTINE_BODY"),
string_column("ROUTINE_DEFINITION"),
string_column("EXTERNAL_NAME"),
string_column("EXTERNAL_LANGUAGE"),
string_column("PARAMETER_STYLE"),
string_column("IS_DETERMINISTIC"),
string_column("SQL_DATA_ACCESS"),
string_column("SQL_PATH"),
string_column("SECURITY_TYPE"),
datetime_column("CREATED"),
datetime_column("LAST_ALTERED"),
string_column("SQL_MODE"),
string_column("ROUTINE_COMMENT"),
string_column("DEFINER"),
string_column("CHARACTER_SET_CLIENT"),
string_column("COLLATION_CONNECTION"),
string_column("DATABASE_COLLATION"),
],
vec![],
),
SCHEMA_PRIVILEGES => (
vec![
string_column("GRANTEE"),
string_column("TABLE_CATALOG"),
string_column("TABLE_SCHEMA"),
string_column("PRIVILEGE_TYPE"),
string_column("IS_GRANTABLE"),
],
vec![],
),
TABLE_PRIVILEGES => (
vec![
string_column("GRANTEE"),
string_column("TABLE_CATALOG"),
string_column("TABLE_SCHEMA"),
string_column("TABLE_NAME"),
string_column("PRIVILEGE_TYPE"),
string_column("IS_GRANTABLE"),
],
vec![],
),
TRIGGERS => (
vec![
string_column("TRIGGER_CATALOG"),
string_column("TRIGGER_SCHEMA"),
string_column("TRIGGER_NAME"),
string_column("EVENT_MANIPULATION"),
string_column("EVENT_OBJECT_CATALOG"),
string_column("EVENT_OBJECT_SCHEMA"),
string_column("EVENT_OBJECT_TABLE"),
bigint_column("ACTION_ORDER"),
string_column("ACTION_CONDITION"),
string_column("ACTION_STATEMENT"),
string_column("ACTION_ORIENTATION"),
string_column("ACTION_TIMING"),
string_column("ACTION_REFERENCE_OLD_TABLE"),
string_column("ACTION_REFERENCE_NEW_TABLE"),
string_column("ACTION_REFERENCE_OLD_ROW"),
string_column("ACTION_REFERENCE_NEW_ROW"),
datetime_column("CREATED"),
string_column("SQL_MODE"),
string_column("DEFINER"),
string_column("CHARACTER_SET_CLIENT"),
string_column("COLLATION_CONNECTION"),
string_column("DATABASE_COLLATION"),
],
vec![],
),
// TODO: Considering store internal metrics in `global_status` and
// `session_status` tables.
GLOBAL_STATUS => (
vec![
string_column("VARIABLE_NAME"),
string_column("VARIABLE_VALUE"),
],
vec![],
),
SESSION_STATUS => (
vec![
string_column("VARIABLE_NAME"),
string_column("VARIABLE_VALUE"),
],
vec![],
),
_ => unreachable!("Unknown table in information_schema: {}", table_name),
};
@@ -115,6 +429,22 @@ fn string_column(name: &str) -> ColumnSchema {
)
}
fn bigint_column(name: &str) -> ColumnSchema {
ColumnSchema::new(
str::to_lowercase(name),
ConcreteDataType::int64_datatype(),
false,
)
}
fn datetime_column(name: &str) -> ColumnSchema {
ColumnSchema::new(
str::to_lowercase(name),
ConcreteDataType::datetime_datatype(),
false,
)
}
#[cfg(test)]
mod tests {
use super::*;

View File

@@ -0,0 +1,609 @@
// 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 arrow::array::StringArray;
use arrow::compute::kernels::comparison;
use common_query::logical_plan::DfExpr;
use datafusion::common::ScalarValue;
use datafusion::logical_expr::expr::Like;
use datafusion::logical_expr::Operator;
use datatypes::value::Value;
use store_api::storage::ScanRequest;
type ColumnName = String;
/// Predicate to filter `information_schema` tables stream,
/// we only support these simple predicates currently.
/// TODO(dennis): supports more predicate types.
#[derive(Clone, PartialEq, Eq, Debug)]
enum Predicate {
Eq(ColumnName, Value),
Like(ColumnName, String, bool),
NotEq(ColumnName, Value),
InList(ColumnName, Vec<Value>),
And(Box<Predicate>, Box<Predicate>),
Or(Box<Predicate>, Box<Predicate>),
Not(Box<Predicate>),
}
impl Predicate {
/// Evaluate the predicate with the row, returns:
/// - `None` when the predicate can't evaluate with the row.
/// - `Some(true)` when the predicate is satisfied,
/// - `Some(false)` when the predicate is not satisfied,
fn eval(&self, row: &[(&str, &Value)]) -> Option<bool> {
match self {
Predicate::Eq(c, v) => {
for (column, value) in row {
if c != column {
continue;
}
return Some(v == *value);
}
}
Predicate::Like(c, pattern, case_insensitive) => {
for (column, value) in row {
if c != column {
continue;
}
let Value::String(bs) = value else {
continue;
};
return like_utf8(bs.as_utf8(), pattern, case_insensitive);
}
}
Predicate::NotEq(c, v) => {
for (column, value) in row {
if c != column {
continue;
}
return Some(v != *value);
}
}
Predicate::InList(c, values) => {
for (column, value) in row {
if c != column {
continue;
}
return Some(values.iter().any(|v| v == *value));
}
}
Predicate::And(left, right) => {
let left = left.eval(row);
// short-circuit
if matches!(left, Some(false)) {
return Some(false);
}
return match (left, right.eval(row)) {
(Some(left), Some(right)) => Some(left && right),
(None, Some(false)) => Some(false),
_ => None,
};
}
Predicate::Or(left, right) => {
let left = left.eval(row);
// short-circuit
if matches!(left, Some(true)) {
return Some(true);
}
return match (left, right.eval(row)) {
(Some(left), Some(right)) => Some(left || right),
(None, Some(true)) => Some(true),
_ => None,
};
}
Predicate::Not(p) => {
let Some(b) = p.eval(row) else {
return None;
};
return Some(!b);
}
}
// Can't evaluate predicate with the row
None
}
/// Try to create a predicate from datafusion [`Expr`], return None if fails.
fn from_expr(expr: DfExpr) -> Option<Predicate> {
match expr {
// NOT expr
DfExpr::Not(expr) => {
let Some(p) = Self::from_expr(*expr) else {
return None;
};
Some(Predicate::Not(Box::new(p)))
}
// expr LIKE pattern
DfExpr::Like(Like {
negated,
expr,
pattern,
case_insensitive,
..
}) if is_column(&expr) && is_string_literal(&pattern) => {
// Safety: ensured by gurad
let DfExpr::Column(c) = *expr else {
unreachable!();
};
let DfExpr::Literal(ScalarValue::Utf8(Some(pattern))) = *pattern else {
unreachable!();
};
let p = Predicate::Like(c.name, pattern, case_insensitive);
if negated {
Some(Predicate::Not(Box::new(p)))
} else {
Some(p)
}
}
// left OP right
DfExpr::BinaryExpr(bin) => match (*bin.left, bin.op, *bin.right) {
// left == right
(DfExpr::Literal(scalar), Operator::Eq, DfExpr::Column(c))
| (DfExpr::Column(c), Operator::Eq, DfExpr::Literal(scalar)) => {
let Ok(v) = Value::try_from(scalar) else {
return None;
};
Some(Predicate::Eq(c.name, v))
}
// left != right
(DfExpr::Literal(scalar), Operator::NotEq, DfExpr::Column(c))
| (DfExpr::Column(c), Operator::NotEq, DfExpr::Literal(scalar)) => {
let Ok(v) = Value::try_from(scalar) else {
return None;
};
Some(Predicate::NotEq(c.name, v))
}
// left AND right
(left, Operator::And, right) => {
let Some(left) = Self::from_expr(left) else {
return None;
};
let Some(right) = Self::from_expr(right) else {
return None;
};
Some(Predicate::And(Box::new(left), Box::new(right)))
}
// left OR right
(left, Operator::Or, right) => {
let Some(left) = Self::from_expr(left) else {
return None;
};
let Some(right) = Self::from_expr(right) else {
return None;
};
Some(Predicate::Or(Box::new(left), Box::new(right)))
}
_ => None,
},
// [NOT] IN (LIST)
DfExpr::InList(list) => {
match (*list.expr, list.list, list.negated) {
// column [NOT] IN (v1, v2, v3, ...)
(DfExpr::Column(c), list, negated) if is_all_scalars(&list) => {
let mut values = Vec::with_capacity(list.len());
for scalar in list {
// Safety: checked by `is_all_scalars`
let DfExpr::Literal(scalar) = scalar else {
unreachable!();
};
let Ok(value) = Value::try_from(scalar) else {
return None;
};
values.push(value);
}
let predicate = Predicate::InList(c.name, values);
if negated {
Some(Predicate::Not(Box::new(predicate)))
} else {
Some(predicate)
}
}
_ => None,
}
}
_ => None,
}
}
}
/// Perform SQL left LIKE right, return `None` if fail to evaluate.
/// - `s` the target string
/// - `pattern` the pattern just like '%abc'
/// - `case_insensitive` whether to perform case-insensitive like or not.
fn like_utf8(s: &str, pattern: &str, case_insensitive: &bool) -> Option<bool> {
let array = StringArray::from(vec![s]);
let patterns = StringArray::new_scalar(pattern);
let Ok(booleans) = (if *case_insensitive {
comparison::ilike(&array, &patterns)
} else {
comparison::like(&array, &patterns)
}) else {
return None;
};
// Safety: at least one value in result
Some(booleans.value(0))
}
fn is_string_literal(expr: &DfExpr) -> bool {
matches!(expr, DfExpr::Literal(ScalarValue::Utf8(Some(_))))
}
fn is_column(expr: &DfExpr) -> bool {
matches!(expr, DfExpr::Column(_))
}
/// A list of predicate
pub struct Predicates {
predicates: Vec<Predicate>,
}
impl Predicates {
/// Try its best to create predicates from [`ScanRequest`].
pub fn from_scan_request(request: &Option<ScanRequest>) -> Predicates {
if let Some(request) = request {
let mut predicates = Vec::with_capacity(request.filters.len());
for filter in &request.filters {
if let Some(predicate) = Predicate::from_expr(filter.df_expr().clone()) {
predicates.push(predicate);
}
}
Self { predicates }
} else {
Self {
predicates: Vec::new(),
}
}
}
/// Evaluate the predicates with the row.
/// returns true when all the predicates are satisfied or can't be evaluated.
pub fn eval(&self, row: &[(&str, &Value)]) -> bool {
// fast path
if self.predicates.is_empty() {
return true;
}
self.predicates
.iter()
.filter_map(|p| p.eval(row))
.all(|b| b)
}
}
/// Returns true when the values are all [`DfExpr::Literal`].
fn is_all_scalars(list: &[DfExpr]) -> bool {
list.iter().all(|v| matches!(v, DfExpr::Literal(_)))
}
#[cfg(test)]
mod tests {
use datafusion::common::{Column, ScalarValue};
use datafusion::logical_expr::expr::InList;
use datafusion::logical_expr::BinaryExpr;
use super::*;
#[test]
fn test_predicate_eval() {
let a_col = "a".to_string();
let b_col = "b".to_string();
let a_value = Value::from("a_value");
let b_value = Value::from("b_value");
let wrong_value = Value::from("wrong_value");
let a_row = [(a_col.as_str(), &a_value)];
let b_row = [("b", &wrong_value)];
let wrong_row = [(a_col.as_str(), &wrong_value)];
// Predicate::Eq
let p = Predicate::Eq(a_col.clone(), a_value.clone());
assert!(p.eval(&a_row).unwrap());
assert!(p.eval(&b_row).is_none());
assert!(!p.eval(&wrong_row).unwrap());
// Predicate::NotEq
let p = Predicate::NotEq(a_col.clone(), a_value.clone());
assert!(!p.eval(&a_row).unwrap());
assert!(p.eval(&b_row).is_none());
assert!(p.eval(&wrong_row).unwrap());
// Predicate::InList
let p = Predicate::InList(a_col.clone(), vec![a_value.clone(), b_value.clone()]);
assert!(p.eval(&a_row).unwrap());
assert!(p.eval(&b_row).is_none());
assert!(!p.eval(&wrong_row).unwrap());
assert!(p.eval(&[(&a_col, &b_value)]).unwrap());
let p1 = Predicate::Eq(a_col.clone(), a_value.clone());
let p2 = Predicate::Eq(b_col.clone(), b_value.clone());
let row = [(a_col.as_str(), &a_value), (b_col.as_str(), &b_value)];
let wrong_row = [(a_col.as_str(), &a_value), (b_col.as_str(), &wrong_value)];
//Predicate::And
let p = Predicate::And(Box::new(p1.clone()), Box::new(p2.clone()));
assert!(p.eval(&row).unwrap());
assert!(!p.eval(&wrong_row).unwrap());
assert!(p.eval(&[]).is_none());
assert!(p.eval(&[("c", &a_value)]).is_none());
assert!(!p
.eval(&[(a_col.as_str(), &b_value), (b_col.as_str(), &a_value)])
.unwrap());
assert!(!p
.eval(&[(a_col.as_str(), &b_value), (b_col.as_str(), &b_value)])
.unwrap());
assert!(p
.eval(&[(a_col.as_ref(), &a_value), ("c", &a_value)])
.is_none());
assert!(!p
.eval(&[(a_col.as_ref(), &b_value), ("c", &a_value)])
.unwrap());
//Predicate::Or
let p = Predicate::Or(Box::new(p1), Box::new(p2));
assert!(p.eval(&row).unwrap());
assert!(p.eval(&wrong_row).unwrap());
assert!(p.eval(&[]).is_none());
assert!(p.eval(&[("c", &a_value)]).is_none());
assert!(!p
.eval(&[(a_col.as_str(), &b_value), (b_col.as_str(), &a_value)])
.unwrap());
assert!(p
.eval(&[(a_col.as_str(), &b_value), (b_col.as_str(), &b_value)])
.unwrap());
assert!(p
.eval(&[(a_col.as_ref(), &a_value), ("c", &a_value)])
.unwrap());
assert!(p
.eval(&[(a_col.as_ref(), &b_value), ("c", &a_value)])
.is_none());
}
#[test]
fn test_predicate_like() {
// case insensitive
let expr = DfExpr::Like(Like {
negated: false,
expr: Box::new(column("a")),
pattern: Box::new(string_literal("%abc")),
case_insensitive: true,
escape_char: None,
});
let p = Predicate::from_expr(expr).unwrap();
assert!(
matches!(&p, Predicate::Like(c, pattern, case_insensitive) if
c == "a"
&& pattern == "%abc"
&& *case_insensitive)
);
let match_row = [
("a", &Value::from("hello AbC")),
("b", &Value::from("b value")),
];
let unmatch_row = [("a", &Value::from("bca")), ("b", &Value::from("b value"))];
assert!(p.eval(&match_row).unwrap());
assert!(!p.eval(&unmatch_row).unwrap());
assert!(p.eval(&[]).is_none());
// case sensitive
let expr = DfExpr::Like(Like {
negated: false,
expr: Box::new(column("a")),
pattern: Box::new(string_literal("%abc")),
case_insensitive: false,
escape_char: None,
});
let p = Predicate::from_expr(expr).unwrap();
assert!(
matches!(&p, Predicate::Like(c, pattern, case_insensitive) if
c == "a"
&& pattern == "%abc"
&& !*case_insensitive)
);
assert!(!p.eval(&match_row).unwrap());
assert!(!p.eval(&unmatch_row).unwrap());
assert!(p.eval(&[]).is_none());
// not like
let expr = DfExpr::Like(Like {
negated: true,
expr: Box::new(column("a")),
pattern: Box::new(string_literal("%abc")),
case_insensitive: true,
escape_char: None,
});
let p = Predicate::from_expr(expr).unwrap();
assert!(!p.eval(&match_row).unwrap());
assert!(p.eval(&unmatch_row).unwrap());
assert!(p.eval(&[]).is_none());
}
fn column(name: &str) -> DfExpr {
DfExpr::Column(Column {
relation: None,
name: name.to_string(),
})
}
fn string_literal(v: &str) -> DfExpr {
DfExpr::Literal(ScalarValue::Utf8(Some(v.to_string())))
}
fn match_string_value(v: &Value, expected: &str) -> bool {
matches!(v, Value::String(bs) if bs.as_utf8() == expected)
}
fn match_string_values(vs: &[Value], expected: &[&str]) -> bool {
assert_eq!(vs.len(), expected.len());
let mut result = true;
for (i, v) in vs.iter().enumerate() {
result = result && match_string_value(v, expected[i]);
}
result
}
fn mock_exprs() -> (DfExpr, DfExpr) {
let expr1 = DfExpr::BinaryExpr(BinaryExpr {
left: Box::new(column("a")),
op: Operator::Eq,
right: Box::new(string_literal("a_value")),
});
let expr2 = DfExpr::BinaryExpr(BinaryExpr {
left: Box::new(column("b")),
op: Operator::NotEq,
right: Box::new(string_literal("b_value")),
});
(expr1, expr2)
}
#[test]
fn test_predicate_from_expr() {
let (expr1, expr2) = mock_exprs();
let p1 = Predicate::from_expr(expr1.clone()).unwrap();
assert!(matches!(&p1, Predicate::Eq(column, v) if column == "a"
&& match_string_value(v, "a_value")));
let p2 = Predicate::from_expr(expr2.clone()).unwrap();
assert!(matches!(&p2, Predicate::NotEq(column, v) if column == "b"
&& match_string_value(v, "b_value")));
let and_expr = DfExpr::BinaryExpr(BinaryExpr {
left: Box::new(expr1.clone()),
op: Operator::And,
right: Box::new(expr2.clone()),
});
let or_expr = DfExpr::BinaryExpr(BinaryExpr {
left: Box::new(expr1.clone()),
op: Operator::Or,
right: Box::new(expr2.clone()),
});
let not_expr = DfExpr::Not(Box::new(expr1.clone()));
let and_p = Predicate::from_expr(and_expr).unwrap();
assert!(matches!(and_p, Predicate::And(left, right) if *left == p1 && *right == p2));
let or_p = Predicate::from_expr(or_expr).unwrap();
assert!(matches!(or_p, Predicate::Or(left, right) if *left == p1 && *right == p2));
let not_p = Predicate::from_expr(not_expr).unwrap();
assert!(matches!(not_p, Predicate::Not(p) if *p == p1));
let inlist_expr = DfExpr::InList(InList {
expr: Box::new(column("a")),
list: vec![string_literal("a1"), string_literal("a2")],
negated: false,
});
let inlist_p = Predicate::from_expr(inlist_expr).unwrap();
assert!(matches!(&inlist_p, Predicate::InList(c, values) if c == "a"
&& match_string_values(values, &["a1", "a2"])));
let inlist_expr = DfExpr::InList(InList {
expr: Box::new(column("a")),
list: vec![string_literal("a1"), string_literal("a2")],
negated: true,
});
let inlist_p = Predicate::from_expr(inlist_expr).unwrap();
assert!(matches!(inlist_p, Predicate::Not(p) if
matches!(&*p,
Predicate::InList(c, values) if c == "a"
&& match_string_values(values, &["a1", "a2"]))));
}
#[test]
fn test_predicates_from_scan_request() {
let predicates = Predicates::from_scan_request(&None);
assert!(predicates.predicates.is_empty());
let (expr1, expr2) = mock_exprs();
let request = ScanRequest {
filters: vec![expr1.into(), expr2.into()],
..Default::default()
};
let predicates = Predicates::from_scan_request(&Some(request));
assert_eq!(2, predicates.predicates.len());
assert!(
matches!(&predicates.predicates[0], Predicate::Eq(column, v) if column == "a"
&& match_string_value(v, "a_value"))
);
assert!(
matches!(&predicates.predicates[1], Predicate::NotEq(column, v) if column == "b"
&& match_string_value(v, "b_value"))
);
}
#[test]
fn test_predicates_eval_row() {
let wrong_row = [
("a", &Value::from("a_value")),
("b", &Value::from("b_value")),
("c", &Value::from("c_value")),
];
let row = [
("a", &Value::from("a_value")),
("b", &Value::from("not_b_value")),
("c", &Value::from("c_value")),
];
let c_row = [("c", &Value::from("c_value"))];
// test empty predicates, always returns true
let predicates = Predicates::from_scan_request(&None);
assert!(predicates.eval(&row));
assert!(predicates.eval(&wrong_row));
assert!(predicates.eval(&c_row));
let (expr1, expr2) = mock_exprs();
let request = ScanRequest {
filters: vec![expr1.into(), expr2.into()],
..Default::default()
};
let predicates = Predicates::from_scan_request(&Some(request));
assert!(predicates.eval(&row));
assert!(!predicates.eval(&wrong_row));
assert!(predicates.eval(&c_row));
}
}

View File

@@ -0,0 +1,250 @@
// 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 arrow_schema::SchemaRef as ArrowSchemaRef;
use common_catalog::consts::INFORMATION_SCHEMA_RUNTIME_METRICS_TABLE_ID;
use common_error::ext::BoxedError;
use common_query::physical_plan::TaskContext;
use common_recordbatch::adapter::RecordBatchStreamAdapter;
use common_recordbatch::{RecordBatch, SendableRecordBatchStream};
use common_time::util::current_time_millis;
use datafusion::physical_plan::stream::RecordBatchStreamAdapter as DfRecordBatchStreamAdapter;
use datafusion::physical_plan::streaming::PartitionStream as DfPartitionStream;
use datafusion::physical_plan::SendableRecordBatchStream as DfSendableRecordBatchStream;
use datatypes::prelude::{ConcreteDataType, MutableVector};
use datatypes::scalars::ScalarVectorBuilder;
use datatypes::schema::{ColumnSchema, Schema, SchemaRef};
use datatypes::vectors::{
ConstantVector, Float64VectorBuilder, StringVector, StringVectorBuilder,
TimestampMillisecondVector, VectorRef,
};
use itertools::Itertools;
use snafu::ResultExt;
use store_api::storage::{ScanRequest, TableId};
use super::{InformationTable, RUNTIME_METRICS};
use crate::error::{CreateRecordBatchSnafu, InternalSnafu, Result};
pub(super) struct InformationSchemaMetrics {
schema: SchemaRef,
}
const METRIC_NAME: &str = "metric_name";
const METRIC_VALUE: &str = "value";
const METRIC_LABELS: &str = "labels";
const NODE: &str = "node";
const NODE_TYPE: &str = "node_type";
const TIMESTAMP: &str = "timestamp";
/// The `information_schema.runtime_metrics` virtual table.
/// It provides the GreptimeDB runtime metrics for the users by SQL.
impl InformationSchemaMetrics {
pub(super) fn new() -> Self {
Self {
schema: Self::schema(),
}
}
fn schema() -> SchemaRef {
Arc::new(Schema::new(vec![
ColumnSchema::new(METRIC_NAME, ConcreteDataType::string_datatype(), false),
ColumnSchema::new(METRIC_VALUE, ConcreteDataType::float64_datatype(), false),
ColumnSchema::new(METRIC_LABELS, ConcreteDataType::string_datatype(), true),
ColumnSchema::new(NODE, ConcreteDataType::string_datatype(), false),
ColumnSchema::new(NODE_TYPE, ConcreteDataType::string_datatype(), false),
ColumnSchema::new(
TIMESTAMP,
ConcreteDataType::timestamp_millisecond_datatype(),
false,
),
]))
}
fn builder(&self) -> InformationSchemaMetricsBuilder {
InformationSchemaMetricsBuilder::new(self.schema.clone())
}
}
impl InformationTable for InformationSchemaMetrics {
fn table_id(&self) -> TableId {
INFORMATION_SCHEMA_RUNTIME_METRICS_TABLE_ID
}
fn table_name(&self) -> &'static str {
RUNTIME_METRICS
}
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
fn to_stream(&self, request: ScanRequest) -> Result<SendableRecordBatchStream> {
let schema = self.schema.arrow_schema().clone();
let mut builder = self.builder();
let stream = Box::pin(DfRecordBatchStreamAdapter::new(
schema,
futures::stream::once(async move {
builder
.make_metrics(Some(request))
.await
.map(|x| x.into_df_record_batch())
.map_err(Into::into)
}),
));
Ok(Box::pin(
RecordBatchStreamAdapter::try_new(stream)
.map_err(BoxedError::new)
.context(InternalSnafu)?,
))
}
}
struct InformationSchemaMetricsBuilder {
schema: SchemaRef,
metric_names: StringVectorBuilder,
metric_values: Float64VectorBuilder,
metric_labels: StringVectorBuilder,
}
impl InformationSchemaMetricsBuilder {
fn new(schema: SchemaRef) -> Self {
Self {
schema,
metric_names: StringVectorBuilder::with_capacity(42),
metric_values: Float64VectorBuilder::with_capacity(42),
metric_labels: StringVectorBuilder::with_capacity(42),
}
}
fn add_metric(&mut self, metric_name: &str, labels: String, metric_value: f64) {
self.metric_names.push(Some(metric_name));
self.metric_values.push(Some(metric_value));
self.metric_labels.push(Some(&labels));
}
async fn make_metrics(&mut self, _request: Option<ScanRequest>) -> Result<RecordBatch> {
let metric_families = prometheus::gather();
let write_request =
common_telemetry::metric::convert_metric_to_write_request(metric_families, None, 0);
for ts in write_request.timeseries {
//Safety: always has `__name__` label
let metric_name = ts
.labels
.iter()
.find_map(|label| {
if label.name == "__name__" {
Some(label.value.clone())
} else {
None
}
})
.unwrap();
self.add_metric(
&metric_name,
ts.labels
.into_iter()
.filter_map(|label| {
if label.name == "__name__" {
None
} else {
Some(format!("{}={}", label.name, label.value))
}
})
.join(", "),
// Safety: always has a sample
ts.samples[0].value,
);
}
self.finish()
}
fn finish(&mut self) -> Result<RecordBatch> {
let rows_num = self.metric_names.len();
let unknowns = Arc::new(ConstantVector::new(
Arc::new(StringVector::from(vec!["unknown"])),
rows_num,
));
let timestamps = Arc::new(ConstantVector::new(
Arc::new(TimestampMillisecondVector::from_slice([
current_time_millis(),
])),
rows_num,
));
let columns: Vec<VectorRef> = vec![
Arc::new(self.metric_names.finish()),
Arc::new(self.metric_values.finish()),
Arc::new(self.metric_labels.finish()),
// TODO(dennis): supports node and node_type for cluster
unknowns.clone(),
unknowns,
timestamps,
];
RecordBatch::new(self.schema.clone(), columns).context(CreateRecordBatchSnafu)
}
}
impl DfPartitionStream for InformationSchemaMetrics {
fn schema(&self) -> &ArrowSchemaRef {
self.schema.arrow_schema()
}
fn execute(&self, _: Arc<TaskContext>) -> DfSendableRecordBatchStream {
let schema = self.schema.arrow_schema().clone();
let mut builder = self.builder();
Box::pin(DfRecordBatchStreamAdapter::new(
schema,
futures::stream::once(async move {
builder
.make_metrics(None)
.await
.map(|x| x.into_df_record_batch())
.map_err(Into::into)
}),
))
}
}
#[cfg(test)]
mod tests {
use common_recordbatch::RecordBatches;
use super::*;
#[tokio::test]
async fn test_make_metrics() {
let metrics = InformationSchemaMetrics::new();
let stream = metrics.to_stream(ScanRequest::default()).unwrap();
let batches = RecordBatches::try_collect(stream).await.unwrap();
let result_literal = batches.pretty_print().unwrap();
assert!(result_literal.contains(METRIC_NAME));
assert!(result_literal.contains(METRIC_VALUE));
assert!(result_literal.contains(METRIC_LABELS));
assert!(result_literal.contains(NODE));
assert!(result_literal.contains(NODE_TYPE));
assert!(result_literal.contains(TIMESTAMP));
}
}

View File

@@ -0,0 +1,228 @@
// 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, Weak};
use arrow_schema::SchemaRef as ArrowSchemaRef;
use common_catalog::consts::INFORMATION_SCHEMA_SCHEMATA_TABLE_ID;
use common_error::ext::BoxedError;
use common_query::physical_plan::TaskContext;
use common_recordbatch::adapter::RecordBatchStreamAdapter;
use common_recordbatch::{RecordBatch, SendableRecordBatchStream};
use datafusion::physical_plan::stream::RecordBatchStreamAdapter as DfRecordBatchStreamAdapter;
use datafusion::physical_plan::streaming::PartitionStream as DfPartitionStream;
use datafusion::physical_plan::SendableRecordBatchStream as DfSendableRecordBatchStream;
use datatypes::prelude::{ConcreteDataType, ScalarVectorBuilder, VectorRef};
use datatypes::schema::{ColumnSchema, Schema, SchemaRef};
use datatypes::value::Value;
use datatypes::vectors::StringVectorBuilder;
use snafu::{OptionExt, ResultExt};
use store_api::storage::{ScanRequest, TableId};
use super::SCHEMATA;
use crate::error::{
CreateRecordBatchSnafu, InternalSnafu, Result, UpgradeWeakCatalogManagerRefSnafu,
};
use crate::information_schema::{InformationTable, Predicates};
use crate::CatalogManager;
const CATALOG_NAME: &str = "catalog_name";
const SCHEMA_NAME: &str = "schema_name";
const DEFAULT_CHARACTER_SET_NAME: &str = "default_character_set_name";
const DEFAULT_COLLATION_NAME: &str = "default_collation_name";
/// The `information_schema.schemata` table implementation.
pub(super) struct InformationSchemaSchemata {
schema: SchemaRef,
catalog_name: String,
catalog_manager: Weak<dyn CatalogManager>,
}
impl InformationSchemaSchemata {
pub(super) fn new(catalog_name: String, catalog_manager: Weak<dyn CatalogManager>) -> Self {
Self {
schema: Self::schema(),
catalog_name,
catalog_manager,
}
}
pub(crate) fn schema() -> SchemaRef {
Arc::new(Schema::new(vec![
ColumnSchema::new(CATALOG_NAME, ConcreteDataType::string_datatype(), false),
ColumnSchema::new(SCHEMA_NAME, ConcreteDataType::string_datatype(), false),
ColumnSchema::new(
DEFAULT_CHARACTER_SET_NAME,
ConcreteDataType::string_datatype(),
false,
),
ColumnSchema::new(
DEFAULT_COLLATION_NAME,
ConcreteDataType::string_datatype(),
false,
),
ColumnSchema::new("sql_path", ConcreteDataType::string_datatype(), true),
]))
}
fn builder(&self) -> InformationSchemaSchemataBuilder {
InformationSchemaSchemataBuilder::new(
self.schema.clone(),
self.catalog_name.clone(),
self.catalog_manager.clone(),
)
}
}
impl InformationTable for InformationSchemaSchemata {
fn table_id(&self) -> TableId {
INFORMATION_SCHEMA_SCHEMATA_TABLE_ID
}
fn table_name(&self) -> &'static str {
SCHEMATA
}
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
fn to_stream(&self, request: ScanRequest) -> Result<SendableRecordBatchStream> {
let schema = self.schema.arrow_schema().clone();
let mut builder = self.builder();
let stream = Box::pin(DfRecordBatchStreamAdapter::new(
schema,
futures::stream::once(async move {
builder
.make_schemata(Some(request))
.await
.map(|x| x.into_df_record_batch())
.map_err(Into::into)
}),
));
Ok(Box::pin(
RecordBatchStreamAdapter::try_new(stream)
.map_err(BoxedError::new)
.context(InternalSnafu)?,
))
}
}
/// Builds the `information_schema.schemata` table row by row
///
/// Columns are based on <https://docs.pingcap.com/tidb/stable/information-schema-schemata>
struct InformationSchemaSchemataBuilder {
schema: SchemaRef,
catalog_name: String,
catalog_manager: Weak<dyn CatalogManager>,
catalog_names: StringVectorBuilder,
schema_names: StringVectorBuilder,
charset_names: StringVectorBuilder,
collation_names: StringVectorBuilder,
sql_paths: StringVectorBuilder,
}
impl InformationSchemaSchemataBuilder {
fn new(
schema: SchemaRef,
catalog_name: String,
catalog_manager: Weak<dyn CatalogManager>,
) -> Self {
Self {
schema,
catalog_name,
catalog_manager,
catalog_names: StringVectorBuilder::with_capacity(42),
schema_names: StringVectorBuilder::with_capacity(42),
charset_names: StringVectorBuilder::with_capacity(42),
collation_names: StringVectorBuilder::with_capacity(42),
sql_paths: StringVectorBuilder::with_capacity(42),
}
}
/// Construct the `information_schema.schemata` virtual table
async fn make_schemata(&mut self, request: Option<ScanRequest>) -> Result<RecordBatch> {
let catalog_name = self.catalog_name.clone();
let catalog_manager = self
.catalog_manager
.upgrade()
.context(UpgradeWeakCatalogManagerRefSnafu)?;
let predicates = Predicates::from_scan_request(&request);
for schema_name in catalog_manager.schema_names(&catalog_name).await? {
if !catalog_manager
.schema_exists(&catalog_name, &schema_name)
.await?
{
continue;
}
self.add_schema(&predicates, &catalog_name, &schema_name);
}
self.finish()
}
fn add_schema(&mut self, predicates: &Predicates, catalog_name: &str, schema_name: &str) {
let row = [
(CATALOG_NAME, &Value::from(catalog_name)),
(SCHEMA_NAME, &Value::from(schema_name)),
(DEFAULT_CHARACTER_SET_NAME, &Value::from("utf8")),
(DEFAULT_COLLATION_NAME, &Value::from("utf8_bin")),
];
if !predicates.eval(&row) {
return;
}
self.catalog_names.push(Some(catalog_name));
self.schema_names.push(Some(schema_name));
self.charset_names.push(Some("utf8"));
self.collation_names.push(Some("utf8_bin"));
self.sql_paths.push(None);
}
fn finish(&mut self) -> Result<RecordBatch> {
let columns: Vec<VectorRef> = vec![
Arc::new(self.catalog_names.finish()),
Arc::new(self.schema_names.finish()),
Arc::new(self.charset_names.finish()),
Arc::new(self.collation_names.finish()),
Arc::new(self.sql_paths.finish()),
];
RecordBatch::new(self.schema.clone(), columns).context(CreateRecordBatchSnafu)
}
}
impl DfPartitionStream for InformationSchemaSchemata {
fn schema(&self) -> &ArrowSchemaRef {
self.schema.arrow_schema()
}
fn execute(&self, _: Arc<TaskContext>) -> DfSendableRecordBatchStream {
let schema = self.schema.arrow_schema().clone();
let mut builder = self.builder();
Box::pin(DfRecordBatchStreamAdapter::new(
schema,
futures::stream::once(async move {
builder
.make_schemata(None)
.await
.map(|x| x.into_df_record_batch())
.map_err(Into::into)
}),
))
}
}

View File

@@ -20,3 +20,22 @@ pub const ENGINES: &str = "engines";
pub const COLUMN_PRIVILEGES: &str = "column_privileges";
pub const COLUMN_STATISTICS: &str = "column_statistics";
pub const BUILD_INFO: &str = "build_info";
pub const CHARACTER_SETS: &str = "character_sets";
pub const COLLATIONS: &str = "collations";
pub const COLLATION_CHARACTER_SET_APPLICABILITY: &str = "collation_character_set_applicability";
pub const CHECK_CONSTRAINTS: &str = "check_constraints";
pub const EVENTS: &str = "events";
pub const FILES: &str = "files";
pub const SCHEMATA: &str = "schemata";
pub const KEY_COLUMN_USAGE: &str = "key_column_usage";
pub const OPTIMIZER_TRACE: &str = "optimizer_trace";
pub const PARAMETERS: &str = "parameters";
pub const PROFILING: &str = "profiling";
pub const REFERENTIAL_CONSTRAINTS: &str = "referential_constraints";
pub const ROUTINES: &str = "routines";
pub const SCHEMA_PRIVILEGES: &str = "schema_privileges";
pub const TABLE_PRIVILEGES: &str = "table_privileges";
pub const TRIGGERS: &str = "triggers";
pub const GLOBAL_STATUS: &str = "global_status";
pub const SESSION_STATUS: &str = "session_status";
pub const RUNTIME_METRICS: &str = "runtime_metrics";

View File

@@ -25,18 +25,26 @@ use datafusion::physical_plan::streaming::PartitionStream as DfPartitionStream;
use datafusion::physical_plan::SendableRecordBatchStream as DfSendableRecordBatchStream;
use datatypes::prelude::{ConcreteDataType, ScalarVectorBuilder, VectorRef};
use datatypes::schema::{ColumnSchema, Schema, SchemaRef};
use datatypes::value::Value;
use datatypes::vectors::{StringVectorBuilder, UInt32VectorBuilder};
use snafu::{OptionExt, ResultExt};
use store_api::storage::TableId;
use store_api::storage::{ScanRequest, TableId};
use table::metadata::TableType;
use super::TABLES;
use crate::error::{
CreateRecordBatchSnafu, InternalSnafu, Result, UpgradeWeakCatalogManagerRefSnafu,
};
use crate::information_schema::InformationTable;
use crate::information_schema::{InformationTable, Predicates};
use crate::CatalogManager;
const TABLE_CATALOG: &str = "table_catalog";
const TABLE_SCHEMA: &str = "table_schema";
const TABLE_NAME: &str = "table_name";
const TABLE_TYPE: &str = "table_type";
const TABLE_ID: &str = "table_id";
const ENGINE: &str = "engine";
pub(super) struct InformationSchemaTables {
schema: SchemaRef,
catalog_name: String,
@@ -54,12 +62,12 @@ impl InformationSchemaTables {
pub(crate) fn schema() -> SchemaRef {
Arc::new(Schema::new(vec![
ColumnSchema::new("table_catalog", ConcreteDataType::string_datatype(), false),
ColumnSchema::new("table_schema", ConcreteDataType::string_datatype(), false),
ColumnSchema::new("table_name", ConcreteDataType::string_datatype(), false),
ColumnSchema::new("table_type", ConcreteDataType::string_datatype(), false),
ColumnSchema::new("table_id", ConcreteDataType::uint32_datatype(), true),
ColumnSchema::new("engine", ConcreteDataType::string_datatype(), true),
ColumnSchema::new(TABLE_CATALOG, ConcreteDataType::string_datatype(), false),
ColumnSchema::new(TABLE_SCHEMA, ConcreteDataType::string_datatype(), false),
ColumnSchema::new(TABLE_NAME, ConcreteDataType::string_datatype(), false),
ColumnSchema::new(TABLE_TYPE, ConcreteDataType::string_datatype(), false),
ColumnSchema::new(TABLE_ID, ConcreteDataType::uint32_datatype(), true),
ColumnSchema::new(ENGINE, ConcreteDataType::string_datatype(), true),
]))
}
@@ -85,14 +93,14 @@ impl InformationTable for InformationSchemaTables {
self.schema.clone()
}
fn to_stream(&self) -> Result<SendableRecordBatchStream> {
fn to_stream(&self, request: ScanRequest) -> Result<SendableRecordBatchStream> {
let schema = self.schema.arrow_schema().clone();
let mut builder = self.builder();
let stream = Box::pin(DfRecordBatchStreamAdapter::new(
schema,
futures::stream::once(async move {
builder
.make_tables()
.make_tables(Some(request))
.await
.map(|x| x.into_df_record_batch())
.map_err(Into::into)
@@ -142,12 +150,13 @@ impl InformationSchemaTablesBuilder {
}
/// Construct the `information_schema.tables` virtual table
async fn make_tables(&mut self) -> Result<RecordBatch> {
async fn make_tables(&mut self, request: Option<ScanRequest>) -> Result<RecordBatch> {
let catalog_name = self.catalog_name.clone();
let catalog_manager = self
.catalog_manager
.upgrade()
.context(UpgradeWeakCatalogManagerRefSnafu)?;
let predicates = Predicates::from_scan_request(&request);
for schema_name in catalog_manager.schema_names(&catalog_name).await? {
if !catalog_manager
@@ -167,6 +176,7 @@ impl InformationSchemaTablesBuilder {
{
let table_info = table.table_info();
self.add_table(
&predicates,
&catalog_name,
&schema_name,
&table_name,
@@ -183,8 +193,10 @@ impl InformationSchemaTablesBuilder {
self.finish()
}
#[allow(clippy::too_many_arguments)]
fn add_table(
&mut self,
predicates: &Predicates,
catalog_name: &str,
schema_name: &str,
table_name: &str,
@@ -192,14 +204,27 @@ impl InformationSchemaTablesBuilder {
table_id: Option<u32>,
engine: Option<&str>,
) {
self.catalog_names.push(Some(catalog_name));
self.schema_names.push(Some(schema_name));
self.table_names.push(Some(table_name));
self.table_types.push(Some(match table_type {
let table_type = match table_type {
TableType::Base => "BASE TABLE",
TableType::View => "VIEW",
TableType::Temporary => "LOCAL TEMPORARY",
}));
};
let row = [
(TABLE_CATALOG, &Value::from(catalog_name)),
(TABLE_SCHEMA, &Value::from(schema_name)),
(TABLE_NAME, &Value::from(table_name)),
(TABLE_TYPE, &Value::from(table_type)),
];
if !predicates.eval(&row) {
return;
}
self.catalog_names.push(Some(catalog_name));
self.schema_names.push(Some(schema_name));
self.table_names.push(Some(table_name));
self.table_types.push(Some(table_type));
self.table_ids.push(table_id);
self.engines.push(engine);
}
@@ -229,7 +254,7 @@ impl DfPartitionStream for InformationSchemaTables {
schema,
futures::stream::once(async move {
builder
.make_tables()
.make_tables(None)
.await
.map(|x| x.into_df_record_batch())
.map_err(Into::into)

View File

@@ -19,17 +19,17 @@ use prometheus::*;
lazy_static! {
pub static ref METRIC_CATALOG_MANAGER_CATALOG_COUNT: IntGauge =
register_int_gauge!("catalog_catalog_count", "catalog catalog count").unwrap();
register_int_gauge!("greptime_catalog_catalog_count", "catalog catalog count").unwrap();
pub static ref METRIC_CATALOG_MANAGER_SCHEMA_COUNT: IntGauge =
register_int_gauge!("catalog_schema_count", "catalog schema count").unwrap();
register_int_gauge!("greptime_catalog_schema_count", "catalog schema count").unwrap();
pub static ref METRIC_CATALOG_MANAGER_TABLE_COUNT: IntGaugeVec = register_int_gauge_vec!(
"catalog_table_count",
"greptime_catalog_table_count",
"catalog table count",
&[METRIC_DB_LABEL]
)
.unwrap();
pub static ref METRIC_CATALOG_KV_REMOTE_GET: Histogram =
register_histogram!("catalog_kv_get_remote", "catalog kv get remote").unwrap();
register_histogram!("greptime_catalog_kv_get_remote", "catalog kv get remote").unwrap();
pub static ref METRIC_CATALOG_KV_GET: Histogram =
register_histogram!("catalog_kv_get", "catalog kv get").unwrap();
register_histogram!("greptime_catalog_kv_get", "catalog kv get").unwrap();
}

View File

@@ -15,7 +15,6 @@
use std::collections::HashMap;
use std::sync::Arc;
use common_catalog::consts::INFORMATION_SCHEMA_NAME;
use common_catalog::format_full_table_name;
use datafusion::common::{ResolvedTableReference, TableReference};
use datafusion::datasource::provider_as_source;
@@ -30,7 +29,7 @@ use crate::CatalogManagerRef;
pub struct DfTableSourceProvider {
catalog_manager: CatalogManagerRef,
resolved_tables: HashMap<String, Arc<dyn TableSource>>,
disallow_cross_schema_query: bool,
disallow_cross_catalog_query: bool,
default_catalog: String,
default_schema: String,
}
@@ -38,12 +37,12 @@ pub struct DfTableSourceProvider {
impl DfTableSourceProvider {
pub fn new(
catalog_manager: CatalogManagerRef,
disallow_cross_schema_query: bool,
disallow_cross_catalog_query: bool,
query_ctx: &QueryContext,
) -> Self {
Self {
catalog_manager,
disallow_cross_schema_query,
disallow_cross_catalog_query,
resolved_tables: HashMap::new(),
default_catalog: query_ctx.current_catalog().to_owned(),
default_schema: query_ctx.current_schema().to_owned(),
@@ -54,29 +53,18 @@ impl DfTableSourceProvider {
&'a self,
table_ref: TableReference<'a>,
) -> Result<ResolvedTableReference<'a>> {
if self.disallow_cross_schema_query {
if self.disallow_cross_catalog_query {
match &table_ref {
TableReference::Bare { .. } => (),
TableReference::Partial { schema, .. } => {
ensure!(
schema.as_ref() == self.default_schema
|| schema.as_ref() == INFORMATION_SCHEMA_NAME,
QueryAccessDeniedSnafu {
catalog: &self.default_catalog,
schema: schema.as_ref(),
}
);
}
TableReference::Partial { .. } => {}
TableReference::Full {
catalog, schema, ..
} => {
ensure!(
catalog.as_ref() == self.default_catalog
&& (schema.as_ref() == self.default_schema
|| schema.as_ref() == INFORMATION_SCHEMA_NAME),
catalog.as_ref() == self.default_catalog,
QueryAccessDeniedSnafu {
catalog: catalog.as_ref(),
schema: schema.as_ref()
schema: schema.as_ref(),
}
);
}
@@ -136,21 +124,21 @@ mod tests {
table: Cow::Borrowed("table_name"),
};
let result = table_provider.resolve_table_ref(table_ref);
let _ = result.unwrap();
assert!(result.is_ok());
let table_ref = TableReference::Partial {
schema: Cow::Borrowed("public"),
table: Cow::Borrowed("table_name"),
};
let result = table_provider.resolve_table_ref(table_ref);
let _ = result.unwrap();
assert!(result.is_ok());
let table_ref = TableReference::Partial {
schema: Cow::Borrowed("wrong_schema"),
table: Cow::Borrowed("table_name"),
};
let result = table_provider.resolve_table_ref(table_ref);
assert!(result.is_err());
assert!(result.is_ok());
let table_ref = TableReference::Full {
catalog: Cow::Borrowed("greptime"),
@@ -158,7 +146,7 @@ mod tests {
table: Cow::Borrowed("table_name"),
};
let result = table_provider.resolve_table_ref(table_ref);
let _ = result.unwrap();
assert!(result.is_ok());
let table_ref = TableReference::Full {
catalog: Cow::Borrowed("wrong_catalog"),
@@ -172,14 +160,15 @@ mod tests {
schema: Cow::Borrowed("information_schema"),
table: Cow::Borrowed("columns"),
};
let _ = table_provider.resolve_table_ref(table_ref).unwrap();
let result = table_provider.resolve_table_ref(table_ref);
assert!(result.is_ok());
let table_ref = TableReference::Full {
catalog: Cow::Borrowed("greptime"),
schema: Cow::Borrowed("information_schema"),
table: Cow::Borrowed("columns"),
};
let _ = table_provider.resolve_table_ref(table_ref).unwrap();
assert!(table_provider.resolve_table_ref(table_ref).is_ok());
let table_ref = TableReference::Full {
catalog: Cow::Borrowed("dummy"),
@@ -187,5 +176,12 @@ mod tests {
table: Cow::Borrowed("columns"),
};
assert!(table_provider.resolve_table_ref(table_ref).is_err());
let table_ref = TableReference::Full {
catalog: Cow::Borrowed("greptime"),
schema: Cow::Borrowed("greptime_private"),
table: Cow::Borrowed("columns"),
};
assert!(table_provider.resolve_table_ref(table_ref).is_ok());
}
}

View File

@@ -16,7 +16,7 @@ use std::any::Any;
use common_error::ext::{BoxedError, ErrorExt};
use common_error::status_code::StatusCode;
use common_error::{GREPTIME_ERROR_CODE, GREPTIME_ERROR_MSG};
use common_error::{GREPTIME_DB_HEADER_ERROR_CODE, GREPTIME_DB_HEADER_ERROR_MSG};
use common_macro::stack_trace_debug;
use snafu::{Location, Snafu};
use tonic::{Code, Status};
@@ -115,7 +115,7 @@ impl From<Status> for Error {
.and_then(|v| String::from_utf8(v.as_bytes().to_vec()).ok())
}
let code = get_metadata_value(&e, GREPTIME_ERROR_CODE)
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)
@@ -125,8 +125,8 @@ impl From<Status> for Error {
})
.unwrap_or(StatusCode::Unknown);
let msg =
get_metadata_value(&e, GREPTIME_ERROR_MSG).unwrap_or_else(|| e.message().to_string());
let msg = get_metadata_value(&e, GREPTIME_DB_HEADER_ERROR_MSG)
.unwrap_or_else(|| e.message().to_string());
Self::Server { code, msg }
}

View File

@@ -17,27 +17,30 @@ use prometheus::*;
lazy_static! {
pub static ref METRIC_GRPC_CREATE_TABLE: Histogram =
register_histogram!("grpc_create_table", "grpc create table").unwrap();
pub static ref METRIC_GRPC_PROMQL_RANGE_QUERY: Histogram =
register_histogram!("grpc_promql_range_query", "grpc promql range query").unwrap();
register_histogram!("greptime_grpc_create_table", "grpc create table").unwrap();
pub static ref METRIC_GRPC_PROMQL_RANGE_QUERY: Histogram = register_histogram!(
"greptime_grpc_promql_range_query",
"grpc promql range query"
)
.unwrap();
pub static ref METRIC_GRPC_INSERT: Histogram =
register_histogram!("grpc_insert", "grpc insert").unwrap();
register_histogram!("greptime_grpc_insert", "grpc insert").unwrap();
pub static ref METRIC_GRPC_DELETE: Histogram =
register_histogram!("grpc_delete", "grpc delete").unwrap();
register_histogram!("greptime_grpc_delete", "grpc delete").unwrap();
pub static ref METRIC_GRPC_SQL: Histogram =
register_histogram!("grpc_sql", "grpc sql").unwrap();
register_histogram!("greptime_grpc_sql", "grpc sql").unwrap();
pub static ref METRIC_GRPC_LOGICAL_PLAN: Histogram =
register_histogram!("grpc_logical_plan", "grpc logical plan").unwrap();
register_histogram!("greptime_grpc_logical_plan", "grpc logical plan").unwrap();
pub static ref METRIC_GRPC_ALTER: Histogram =
register_histogram!("grpc_alter", "grpc alter").unwrap();
register_histogram!("greptime_grpc_alter", "grpc alter").unwrap();
pub static ref METRIC_GRPC_DROP_TABLE: Histogram =
register_histogram!("grpc_drop_table", "grpc drop table").unwrap();
register_histogram!("greptime_grpc_drop_table", "grpc drop table").unwrap();
pub static ref METRIC_GRPC_TRUNCATE_TABLE: Histogram =
register_histogram!("grpc_truncate_table", "grpc truncate table").unwrap();
register_histogram!("greptime_grpc_truncate_table", "grpc truncate table").unwrap();
pub static ref METRIC_GRPC_DO_GET: Histogram =
register_histogram!("grpc_do_get", "grpc do get").unwrap();
register_histogram!("greptime_grpc_do_get", "grpc do get").unwrap();
pub static ref METRIC_REGION_REQUEST_GRPC: HistogramVec = register_histogram_vec!(
"grpc_region_request",
"greptime_grpc_region_request",
"grpc region request",
&["request_type"]
)

View File

@@ -39,7 +39,7 @@ use crate::from_grpc_response;
/// ```
///
/// If you want to see a concrete usage example, please see
/// [stream_inserter.rs](https://github.com/GreptimeTeam/greptimedb/blob/develop/src/client/examples/stream_ingest.rs).
/// [stream_inserter.rs](https://github.com/GreptimeTeam/greptimedb/blob/main/src/client/examples/stream_ingest.rs).
pub struct StreamInserter {
sender: mpsc::Sender<GreptimeRequest>,

View File

@@ -28,6 +28,7 @@ use frontend::heartbeat::handler::invalidate_table_cache::InvalidateTableCacheHa
use frontend::heartbeat::HeartbeatTask;
use frontend::instance::builder::FrontendBuilder;
use frontend::instance::{FrontendInstance, Instance as FeInstance};
use frontend::server::Services;
use meta_client::MetaClientOptions;
use servers::tls::{TlsMode, TlsOption};
use servers::Mode;
@@ -246,18 +247,18 @@ impl StartCommand {
meta_client,
)
.with_cache_invalidator(meta_backend)
.with_plugin(plugins)
.with_plugin(plugins.clone())
.with_heartbeat_task(heartbeat_task)
.try_build()
.await
.context(StartFrontendSnafu)?;
instance
.build_export_metrics_task(&opts.export_metrics)
let servers = Services::new(plugins)
.build(opts.clone(), Arc::new(instance.clone()))
.await
.context(StartFrontendSnafu)?;
instance
.build_servers(opts)
.build_servers(opts, servers)
.await
.context(StartFrontendSnafu)?;

View File

@@ -28,7 +28,7 @@ pub mod standalone;
lazy_static::lazy_static! {
static ref APP_VERSION: prometheus::IntGaugeVec =
prometheus::register_int_gauge_vec!("app_version", "app version", &["short_version", "version"]).unwrap();
prometheus::register_int_gauge_vec!("greptime_app_version", "app version", &["short_version", "version"]).unwrap();
}
#[async_trait]

View File

@@ -128,7 +128,7 @@ impl StartCommand {
let mut opts: MetaSrvOptions = Options::load_layered_options(
self.config_file.as_deref(),
self.env_prefix.as_ref(),
None,
MetaSrvOptions::env_list_keys(),
)?;
if let Some(dir) = &cli_options.log_dir {

View File

@@ -22,7 +22,8 @@ use common_config::wal::StandaloneWalConfig;
use common_config::{metadata_store_dir, KvBackendConfig};
use common_meta::cache_invalidator::DummyCacheInvalidator;
use common_meta::datanode_manager::DatanodeManagerRef;
use common_meta::ddl::{DdlTaskExecutorRef, TableMetadataAllocatorRef};
use common_meta::ddl::table_meta::TableMetadataAllocator;
use common_meta::ddl::DdlTaskExecutorRef;
use common_meta::ddl_manager::DdlManager;
use common_meta::key::{TableMetadataManager, TableMetadataManagerRef};
use common_meta::kv_backend::KvBackendRef;
@@ -38,8 +39,8 @@ use datanode::datanode::{Datanode, DatanodeBuilder};
use file_engine::config::EngineConfig as FileEngineConfig;
use frontend::frontend::FrontendOptions;
use frontend::instance::builder::FrontendBuilder;
use frontend::instance::standalone::StandaloneTableMetadataAllocator;
use frontend::instance::{FrontendInstance, Instance as FeInstance, StandaloneDatanodeManager};
use frontend::server::Services;
use frontend::service_config::{
GrpcOptions, InfluxdbOptions, MysqlOptions, OpentsdbOptions, PostgresOptions, PromStoreOptions,
};
@@ -118,6 +119,12 @@ pub struct StandaloneOptions {
pub export_metrics: ExportMetricsOption,
}
impl StandaloneOptions {
pub fn env_list_keys() -> Option<&'static [&'static str]> {
Some(&["wal.broker_endpoints"])
}
}
impl Default for StandaloneOptions {
fn default() -> Self {
Self {
@@ -267,7 +274,7 @@ impl StartCommand {
let opts: StandaloneOptions = Options::load_layered_options(
self.config_file.as_deref(),
self.env_prefix.as_ref(),
None,
StandaloneOptions::env_list_keys(),
)?;
self.convert_options(cli_options, opts)
@@ -406,13 +413,18 @@ impl StartCommand {
opts.wal_meta.clone(),
kv_backend.clone(),
));
let table_meta_allocator = Arc::new(StandaloneTableMetadataAllocator::new(
let table_metadata_manager =
Self::create_table_metadata_manager(kv_backend.clone()).await?;
let table_meta_allocator = TableMetadataAllocator::new(
table_id_sequence,
wal_options_allocator.clone(),
));
table_metadata_manager.clone(),
);
let ddl_task_executor = Self::create_ddl_task_executor(
kv_backend.clone(),
table_metadata_manager,
procedure_manager.clone(),
datanode_manager.clone(),
table_meta_allocator,
@@ -420,17 +432,17 @@ impl StartCommand {
.await?;
let mut frontend = FrontendBuilder::new(kv_backend, datanode_manager, ddl_task_executor)
.with_plugin(fe_plugins)
.with_plugin(fe_plugins.clone())
.try_build()
.await
.context(StartFrontendSnafu)?;
frontend
.build_export_metrics_task(&opts.frontend.export_metrics)
let servers = Services::new(fe_plugins)
.build(opts.clone(), Arc::new(frontend.clone()))
.await
.context(StartFrontendSnafu)?;
frontend
.build_servers(opts)
.build_servers(opts, servers)
.await
.context(StartFrontendSnafu)?;
@@ -443,14 +455,11 @@ impl StartCommand {
}
pub async fn create_ddl_task_executor(
kv_backend: KvBackendRef,
table_metadata_manager: TableMetadataManagerRef,
procedure_manager: ProcedureManagerRef,
datanode_manager: DatanodeManagerRef,
table_meta_allocator: TableMetadataAllocatorRef,
table_meta_allocator: TableMetadataAllocator,
) -> Result<DdlTaskExecutorRef> {
let table_metadata_manager =
Self::create_table_metadata_manager(kv_backend.clone()).await?;
let ddl_task_executor: DdlTaskExecutorRef = Arc::new(
DdlManager::try_new(
procedure_manager,
@@ -466,7 +475,7 @@ impl StartCommand {
Ok(ddl_task_executor)
}
async fn create_table_metadata_manager(
pub async fn create_table_metadata_manager(
kv_backend: KvBackendRef,
) -> Result<TableMetadataManagerRef> {
let table_metadata_manager = Arc::new(TableMetadataManager::new(kv_backend));

View File

@@ -44,6 +44,44 @@ pub const INFORMATION_SCHEMA_COLUMN_PRIVILEGES_TABLE_ID: u32 = 6;
pub const INFORMATION_SCHEMA_COLUMN_STATISTICS_TABLE_ID: u32 = 7;
/// id for information_schema.build_info
pub const INFORMATION_SCHEMA_BUILD_INFO_TABLE_ID: u32 = 8;
/// id for information_schema.CHARACTER_SETS
pub const INFORMATION_SCHEMA_CHARACTER_SETS_TABLE_ID: u32 = 9;
/// id for information_schema.COLLATIONS
pub const INFORMATION_SCHEMA_COLLATIONS_TABLE_ID: u32 = 10;
/// id for information_schema.COLLATIONS
pub const INFORMATION_SCHEMA_COLLATION_CHARACTER_SET_APPLICABILITY_TABLE_ID: u32 = 11;
/// id for information_schema.CHECK_CONSTRAINTS
pub const INFORMATION_SCHEMA_CHECK_CONSTRAINTS_TABLE_ID: u32 = 12;
/// id for information_schema.EVENTS
pub const INFORMATION_SCHEMA_EVENTS_TABLE_ID: u32 = 13;
/// id for information_schema.FILES
pub const INFORMATION_SCHEMA_FILES_TABLE_ID: u32 = 14;
/// id for information_schema.SCHEMATA
pub const INFORMATION_SCHEMA_SCHEMATA_TABLE_ID: u32 = 15;
/// id for information_schema.KEY_COLUMN_USAGE
pub const INFORMATION_SCHEMA_KEY_COLUMN_USAGE_TABLE_ID: u32 = 16;
/// id for information_schema.OPTIMIZER_TRACE
pub const INFORMATION_SCHEMA_OPTIMIZER_TRACE_TABLE_ID: u32 = 17;
/// id for information_schema.PARAMETERS
pub const INFORMATION_SCHEMA_PARAMETERS_TABLE_ID: u32 = 18;
/// id for information_schema.PROFILING
pub const INFORMATION_SCHEMA_PROFILING_TABLE_ID: u32 = 19;
/// id for information_schema.REFERENTIAL_CONSTRAINTS
pub const INFORMATION_SCHEMA_REFERENTIAL_CONSTRAINTS_TABLE_ID: u32 = 20;
/// id for information_schema.ROUTINES
pub const INFORMATION_SCHEMA_ROUTINES_TABLE_ID: u32 = 21;
/// id for information_schema.SCHEMA_PRIVILEGES
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
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;
/// id for information_schema.SESSION_STATUS
pub const INFORMATION_SCHEMA_SESSION_STATUS_TABLE_ID: u32 = 26;
/// id for information_schema.RUNTIME_METRICS
pub const INFORMATION_SCHEMA_RUNTIME_METRICS_TABLE_ID: u32 = 27;
/// ----- End of information_schema tables -----
pub const MITO_ENGINE: &str = "mito";

View File

@@ -17,6 +17,11 @@ use consts::DEFAULT_CATALOG_NAME;
pub mod consts;
pub mod error;
#[inline]
pub fn format_schema_name(catalog: &str, schema: &str) -> String {
format!("{catalog}.{schema}")
}
/// Formats table fully-qualified name
#[inline]
pub fn format_full_table_name(catalog: &str, schema: &str, table: &str) -> String {

View File

@@ -18,9 +18,7 @@ pub mod raft_engine;
use serde::{Deserialize, Serialize};
use serde_with::with_prefix;
pub use crate::wal::kafka::{
KafkaConfig, KafkaOptions as KafkaWalOptions, StandaloneKafkaConfig, Topic as KafkaWalTopic,
};
pub use crate::wal::kafka::{KafkaConfig, KafkaOptions as KafkaWalOptions, StandaloneKafkaConfig};
pub use crate::wal::raft_engine::RaftEngineConfig;
/// An encoded wal options will be wrapped into a (WAL_OPTIONS_KEY, encoded wal options) key-value pair
@@ -90,11 +88,12 @@ mod tests {
#[test]
fn test_serde_kafka_config() {
// With all fields.
let toml_str = r#"
broker_endpoints = ["127.0.0.1:9092"]
max_batch_size = "4MB"
max_batch_size = "1MB"
linger = "200ms"
produce_record_timeout = "100ms"
consumer_wait_timeout = "100ms"
backoff_init = "500ms"
backoff_max = "10s"
backoff_base = 2
@@ -104,9 +103,9 @@ mod tests {
let expected = KafkaConfig {
broker_endpoints: vec!["127.0.0.1:9092".to_string()],
compression: RsKafkaCompression::default(),
max_batch_size: ReadableSize::mb(4),
max_batch_size: ReadableSize::mb(1),
linger: Duration::from_millis(200),
produce_record_timeout: Duration::from_millis(100),
consumer_wait_timeout: Duration::from_millis(100),
backoff: KafkaBackoffConfig {
init: Duration::from_millis(500),
max: Duration::from_secs(10),
@@ -115,6 +114,19 @@ mod tests {
},
};
assert_eq!(decoded, expected);
// With some fields missing.
let toml_str = r#"
broker_endpoints = ["127.0.0.1:9092"]
linger = "200ms"
"#;
let decoded: KafkaConfig = toml::from_str(toml_str).unwrap();
let expected = KafkaConfig {
broker_endpoints: vec!["127.0.0.1:9092".to_string()],
linger: Duration::from_millis(200),
..Default::default()
};
assert_eq!(decoded, expected);
}
#[test]

View File

@@ -19,11 +19,6 @@ use rskafka::client::partition::Compression as RsKafkaCompression;
use serde::{Deserialize, Serialize};
use serde_with::with_prefix;
/// Topic name prefix.
pub const TOPIC_NAME_PREFIX: &str = "greptimedb_wal_topic";
/// Kafka wal topic.
pub type Topic = String;
/// The type of the topic selector, i.e. with which strategy to select a topic.
#[derive(Default, Debug, Clone, Serialize, Deserialize, PartialEq, Eq)]
#[serde(rename_all = "snake_case")]
@@ -40,16 +35,15 @@ pub struct KafkaConfig {
pub broker_endpoints: Vec<String>,
/// The compression algorithm used to compress log entries.
#[serde(skip)]
#[serde(default)]
pub compression: RsKafkaCompression,
/// The maximum log size a kakfa batch producer could buffer.
/// The max size of a single producer batch.
pub max_batch_size: ReadableSize,
/// The linger duration of a kafka batch producer.
#[serde(with = "humantime_serde")]
pub linger: Duration,
/// The maximum amount of time (in milliseconds) to wait for Kafka records to be returned.
/// The consumer wait timeout.
#[serde(with = "humantime_serde")]
pub produce_record_timeout: Duration,
pub consumer_wait_timeout: Duration,
/// The backoff config.
#[serde(flatten, with = "kafka_backoff")]
pub backoff: KafkaBackoffConfig,
@@ -60,9 +54,10 @@ impl Default for KafkaConfig {
Self {
broker_endpoints: vec!["127.0.0.1:9092".to_string()],
compression: RsKafkaCompression::NoCompression,
max_batch_size: ReadableSize::mb(4),
// Warning: Kafka has a default limit of 1MB per message in a topic.
max_batch_size: ReadableSize::mb(1),
linger: Duration::from_millis(200),
produce_record_timeout: Duration::from_millis(100),
consumer_wait_timeout: Duration::from_millis(100),
backoff: KafkaBackoffConfig::default(),
}
}
@@ -73,17 +68,15 @@ with_prefix!(pub kafka_backoff "backoff_");
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)]
#[serde(default)]
pub struct KafkaBackoffConfig {
/// The initial backoff for kafka clients.
/// The initial backoff delay.
#[serde(with = "humantime_serde")]
pub init: Duration,
/// The maximum backoff for kafka clients.
/// The maximum backoff delay.
#[serde(with = "humantime_serde")]
pub max: Duration,
/// Exponential backoff rate, i.e. next backoff = base * current backoff.
// Sets to u32 type since some structs containing the KafkaConfig need to derive the Eq trait.
pub base: u32,
/// Stop reconnecting if the total wait time reaches the deadline.
/// If it's None, the reconnecting won't terminate.
/// The deadline of retries. `None` stands for no deadline.
#[serde(with = "humantime_serde")]
pub deadline: Option<Duration>,
}
@@ -114,7 +107,7 @@ pub struct StandaloneKafkaConfig {
pub num_partitions: i32,
/// The replication factor of each topic.
pub replication_factor: i16,
/// Above which a topic creation operation will be cancelled.
/// The timeout of topic creation.
#[serde(with = "humantime_serde")]
pub create_topic_timeout: Duration,
}
@@ -140,5 +133,5 @@ impl Default for StandaloneKafkaConfig {
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)]
pub struct KafkaOptions {
/// Kafka wal topic.
pub topic: Topic,
pub topic: String,
}

View File

@@ -34,6 +34,13 @@ pub struct RaftEngineConfig {
pub read_batch_size: usize,
// whether to sync log file after every write
pub sync_write: bool,
// whether to reuse logically truncated log files.
pub enable_log_recycle: bool,
// whether to pre-create log files on start up
pub prefill_log_files: bool,
// duration for fsyncing log files.
#[serde(with = "humantime_serde")]
pub sync_period: Option<Duration>,
}
impl Default for RaftEngineConfig {
@@ -45,6 +52,9 @@ impl Default for RaftEngineConfig {
purge_interval: Duration::from_secs(600),
read_batch_size: 128,
sync_write: false,
enable_log_recycle: true,
prefill_log_files: false,
sync_period: None,
}
}
}

View File

@@ -19,7 +19,7 @@ pub mod format;
pub mod mock;
pub mod status_code;
pub const GREPTIME_ERROR_CODE: &str = "x-greptime-err-code";
pub const GREPTIME_ERROR_MSG: &str = "x-greptime-err-msg";
pub const GREPTIME_DB_HEADER_ERROR_CODE: &str = "x-greptime-err-code";
pub const GREPTIME_DB_HEADER_ERROR_MSG: &str = "x-greptime-err-msg";
pub use snafu;

View File

@@ -12,6 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
mod modulo;
mod pow;
mod rate;
@@ -30,11 +31,13 @@ use snafu::ResultExt;
use crate::function::{Function, FunctionContext};
use crate::function_registry::FunctionRegistry;
use crate::scalars::math::modulo::ModuloFunction;
pub(crate) struct MathFunction;
impl MathFunction {
pub fn register(registry: &FunctionRegistry) {
registry.register(Arc::new(ModuloFunction));
registry.register(Arc::new(PowFunction));
registry.register(Arc::new(RateFunction));
registry.register(Arc::new(RangeFunction))

View File

@@ -0,0 +1,241 @@
// 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::fmt;
use std::fmt::Display;
use common_query::error;
use common_query::error::{ArrowComputeSnafu, InvalidFuncArgsSnafu, Result};
use common_query::prelude::{Signature, Volatility};
use datatypes::arrow::compute;
use datatypes::arrow::compute::kernels::numeric;
use datatypes::arrow::datatypes::DataType as ArrowDataType;
use datatypes::prelude::ConcreteDataType;
use datatypes::vectors::{Helper, VectorRef};
use snafu::{ensure, ResultExt};
use crate::function::{Function, FunctionContext};
const NAME: &str = "mod";
/// The function to find remainders
#[derive(Clone, Debug, Default)]
pub struct ModuloFunction;
impl Display for ModuloFunction {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
write!(f, "{}", NAME.to_ascii_uppercase())
}
}
impl Function for ModuloFunction {
fn name(&self) -> &str {
NAME
}
fn return_type(&self, input_types: &[ConcreteDataType]) -> Result<ConcreteDataType> {
if input_types.iter().all(ConcreteDataType::is_signed) {
Ok(ConcreteDataType::int64_datatype())
} else if input_types.iter().all(ConcreteDataType::is_unsigned) {
Ok(ConcreteDataType::uint64_datatype())
} else {
Ok(ConcreteDataType::float64_datatype())
}
}
fn signature(&self) -> Signature {
Signature::uniform(2, ConcreteDataType::numerics(), Volatility::Immutable)
}
fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result<VectorRef> {
ensure!(
columns.len() == 2,
InvalidFuncArgsSnafu {
err_msg: format!(
"The length of the args is not correct, expect exactly two, have: {}",
columns.len()
),
}
);
let nums = &columns[0];
let divs = &columns[1];
let nums_arrow_array = &nums.to_arrow_array();
let divs_arrow_array = &divs.to_arrow_array();
let array = numeric::rem(nums_arrow_array, divs_arrow_array).context(ArrowComputeSnafu)?;
let result = match nums.data_type() {
ConcreteDataType::Int8(_)
| ConcreteDataType::Int16(_)
| ConcreteDataType::Int32(_)
| ConcreteDataType::Int64(_) => compute::cast(&array, &ArrowDataType::Int64),
ConcreteDataType::UInt8(_)
| ConcreteDataType::UInt16(_)
| ConcreteDataType::UInt32(_)
| ConcreteDataType::UInt64(_) => compute::cast(&array, &ArrowDataType::UInt64),
ConcreteDataType::Float32(_) | ConcreteDataType::Float64(_) => {
compute::cast(&array, &ArrowDataType::Float64)
}
_ => unreachable!("unexpected datatype: {:?}", nums.data_type()),
}
.context(ArrowComputeSnafu)?;
Helper::try_into_vector(&result).context(error::FromArrowArraySnafu)
}
}
#[cfg(test)]
mod tests {
use std::sync::Arc;
use common_error::ext::ErrorExt;
use datatypes::value::Value;
use datatypes::vectors::{Float64Vector, Int32Vector, StringVector, UInt32Vector};
use super::*;
#[test]
fn test_mod_function_signed() {
let function = ModuloFunction;
assert_eq!("mod", function.name());
assert_eq!(
ConcreteDataType::int64_datatype(),
function
.return_type(&[ConcreteDataType::int64_datatype()])
.unwrap()
);
assert_eq!(
ConcreteDataType::int64_datatype(),
function
.return_type(&[ConcreteDataType::int32_datatype()])
.unwrap()
);
let nums = vec![18, -17, 5, -6];
let divs = vec![4, 8, -5, -5];
let args: Vec<VectorRef> = vec![
Arc::new(Int32Vector::from_vec(nums.clone())),
Arc::new(Int32Vector::from_vec(divs.clone())),
];
let result = function.eval(FunctionContext::default(), &args).unwrap();
assert_eq!(result.len(), 4);
for i in 0..3 {
let p: i64 = (nums[i] % divs[i]) as i64;
assert!(matches!(result.get(i), Value::Int64(v) if v == p));
}
}
#[test]
fn test_mod_function_unsigned() {
let function = ModuloFunction;
assert_eq!("mod", function.name());
assert_eq!(
ConcreteDataType::uint64_datatype(),
function
.return_type(&[ConcreteDataType::uint64_datatype()])
.unwrap()
);
assert_eq!(
ConcreteDataType::uint64_datatype(),
function
.return_type(&[ConcreteDataType::uint32_datatype()])
.unwrap()
);
let nums: Vec<u32> = vec![18, 17, 5, 6];
let divs: Vec<u32> = vec![4, 8, 5, 5];
let args: Vec<VectorRef> = vec![
Arc::new(UInt32Vector::from_vec(nums.clone())),
Arc::new(UInt32Vector::from_vec(divs.clone())),
];
let result = function.eval(FunctionContext::default(), &args).unwrap();
assert_eq!(result.len(), 4);
for i in 0..3 {
let p: u64 = (nums[i] % divs[i]) as u64;
assert!(matches!(result.get(i), Value::UInt64(v) if v == p));
}
}
#[test]
fn test_mod_function_float() {
let function = ModuloFunction;
assert_eq!("mod", function.name());
assert_eq!(
ConcreteDataType::float64_datatype(),
function
.return_type(&[ConcreteDataType::float64_datatype()])
.unwrap()
);
assert_eq!(
ConcreteDataType::float64_datatype(),
function
.return_type(&[ConcreteDataType::float32_datatype()])
.unwrap()
);
let nums = vec![18.0, 17.0, 5.0, 6.0];
let divs = vec![4.0, 8.0, 5.0, 5.0];
let args: Vec<VectorRef> = vec![
Arc::new(Float64Vector::from_vec(nums.clone())),
Arc::new(Float64Vector::from_vec(divs.clone())),
];
let result = function.eval(FunctionContext::default(), &args).unwrap();
assert_eq!(result.len(), 4);
for i in 0..3 {
let p: f64 = nums[i] % divs[i];
assert!(matches!(result.get(i), Value::Float64(v) if v == p));
}
}
#[test]
fn test_mod_function_errors() {
let function = ModuloFunction;
assert_eq!("mod", function.name());
let nums = vec![27];
let divs = vec![0];
let args: Vec<VectorRef> = vec![
Arc::new(Int32Vector::from_vec(nums.clone())),
Arc::new(Int32Vector::from_vec(divs.clone())),
];
let result = function.eval(FunctionContext::default(), &args);
assert!(result.is_err());
let err_msg = result.unwrap_err().output_msg();
assert_eq!(
err_msg,
"Failed to perform compute operation on arrow arrays: Divide by zero error"
);
let nums = vec![27];
let args: Vec<VectorRef> = vec![Arc::new(Int32Vector::from_vec(nums.clone()))];
let result = function.eval(FunctionContext::default(), &args);
assert!(result.is_err());
let err_msg = result.unwrap_err().output_msg();
assert!(
err_msg.contains("The length of the args is not correct, expect exactly two, have: 1")
);
let nums = vec!["27"];
let divs = vec!["4"];
let args: Vec<VectorRef> = vec![
Arc::new(StringVector::from(nums.clone())),
Arc::new(StringVector::from(divs.clone())),
];
let result = function.eval(FunctionContext::default(), &args);
assert!(result.is_err());
let err_msg = result.unwrap_err().output_msg();
assert!(err_msg.contains("Invalid arithmetic operation"));
}
}

View File

@@ -13,10 +13,12 @@
// limitations under the License.
pub mod build;
pub mod version;
use std::sync::Arc;
use build::BuildFunction;
use version::VersionFunction;
use crate::function_registry::FunctionRegistry;
@@ -25,5 +27,6 @@ pub(crate) struct SystemFunction;
impl SystemFunction {
pub fn register(registry: &FunctionRegistry) {
registry.register(Arc::new(BuildFunction));
registry.register(Arc::new(VersionFunction));
}
}

View File

@@ -0,0 +1,54 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use std::sync::Arc;
use std::{env, fmt};
use common_query::error::Result;
use common_query::prelude::{Signature, Volatility};
use datatypes::data_type::ConcreteDataType;
use datatypes::vectors::{StringVector, VectorRef};
use crate::function::{Function, FunctionContext};
#[derive(Clone, Debug, Default)]
pub(crate) struct VersionFunction;
impl fmt::Display for VersionFunction {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
write!(f, "VERSION")
}
}
impl Function for VersionFunction {
fn name(&self) -> &str {
"version"
}
fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result<ConcreteDataType> {
Ok(ConcreteDataType::string_datatype())
}
fn signature(&self) -> Signature {
Signature::exact(vec![], Volatility::Immutable)
}
fn eval(&self, _func_ctx: FunctionContext, _columns: &[VectorRef]) -> Result<VectorRef> {
let result = StringVector::from(vec![format!(
"5.7.20-greptimedb-{}",
env!("CARGO_PKG_VERSION")
)]);
Ok(Arc::new(result))
}
}

View File

@@ -14,6 +14,7 @@ async-stream.workspace = true
async-trait.workspace = true
base64.workspace = true
bytes.workspace = true
chrono.workspace = true
common-catalog.workspace = true
common-config.workspace = true
common-error.workspace = true
@@ -27,6 +28,7 @@ common-time.workspace = true
datatypes.workspace = true
derive_builder.workspace = true
etcd-client.workspace = true
futures-util.workspace = true
futures.workspace = true
humantime-serde.workspace = true
lazy_static.workspace = true
@@ -51,3 +53,4 @@ chrono.workspace = true
common-procedure = { workspace = true, features = ["testing"] }
datatypes.workspace = true
hyper = { version = "0.14", features = ["full"] }
uuid.workspace = true

View File

@@ -24,11 +24,12 @@ use crate::error::Result;
use crate::key::table_route::TableRouteValue;
use crate::key::TableMetadataManagerRef;
use crate::region_keeper::MemoryRegionKeeperRef;
use crate::rpc::ddl::{CreateTableTask, SubmitDdlTaskRequest, SubmitDdlTaskResponse};
use crate::rpc::ddl::{SubmitDdlTaskRequest, SubmitDdlTaskResponse};
pub mod alter_table;
pub mod create_table;
pub mod drop_table;
pub mod table_meta;
pub mod truncate_table;
pub mod utils;
@@ -64,17 +65,6 @@ pub struct TableMetadata {
pub region_wal_options: HashMap<RegionNumber, String>,
}
#[async_trait::async_trait]
pub trait TableMetadataAllocator: Send + Sync {
async fn create(
&self,
ctx: &TableMetadataAllocatorContext,
task: &CreateTableTask,
) -> Result<TableMetadata>;
}
pub type TableMetadataAllocatorRef = Arc<dyn TableMetadataAllocator>;
#[derive(Clone)]
pub struct DdlContext {
pub datanode_manager: DatanodeManagerRef,

View File

@@ -24,7 +24,7 @@ use async_trait::async_trait;
use common_grpc_expr::alter_expr_to_request;
use common_procedure::error::{FromJsonSnafu, Result as ProcedureResult, ToJsonSnafu};
use common_procedure::{
Context as ProcedureContext, Error as ProcedureError, LockKey, Procedure, Status,
Context as ProcedureContext, Error as ProcedureError, LockKey, Procedure, Status, StringKey,
};
use common_telemetry::tracing_context::TracingContext;
use common_telemetry::{debug, info};
@@ -40,12 +40,11 @@ use table::requests::AlterKind;
use crate::cache_invalidator::Context;
use crate::ddl::utils::handle_operate_region_error;
use crate::ddl::DdlContext;
use crate::error::{
self, ConvertAlterTableRequestSnafu, InvalidProtoMsgSnafu, Result, TableRouteNotFoundSnafu,
};
use crate::error::{self, ConvertAlterTableRequestSnafu, Error, InvalidProtoMsgSnafu, Result};
use crate::key::table_info::TableInfoValue;
use crate::key::table_name::TableNameKey;
use crate::key::DeserializedValueWithBytes;
use crate::lock_key::{CatalogLock, SchemaLock, TableLock, TableNameLock};
use crate::metrics;
use crate::rpc::ddl::AlterTableTask;
use crate::rpc::router::{find_leader_regions, find_leaders};
@@ -65,6 +64,7 @@ impl AlterTableProcedure {
cluster_id: u64,
task: AlterTableTask,
table_info_value: DeserializedValueWithBytes<TableInfoValue>,
physical_table_info: Option<(TableId, TableName)>,
context: DdlContext,
) -> Result<Self> {
let alter_kind = task
@@ -84,7 +84,13 @@ impl AlterTableProcedure {
Ok(Self {
context,
data: AlterTableData::new(task, table_info_value, cluster_id, next_column_id),
data: AlterTableData::new(
task,
table_info_value,
physical_table_info,
cluster_id,
next_column_id,
),
kind,
})
}
@@ -182,23 +188,19 @@ impl AlterTableProcedure {
pub async fn submit_alter_region_requests(&mut self) -> Result<Status> {
let table_id = self.data.table_id();
let table_route = self
let (_, physical_table_route) = self
.context
.table_metadata_manager
.table_route_manager()
.get(table_id)
.await?
.context(TableRouteNotFoundSnafu { table_id })?
.into_inner();
let region_routes = table_route.region_routes();
.get_physical_table_route(table_id)
.await?;
let leaders = find_leaders(region_routes);
let leaders = find_leaders(&physical_table_route.region_routes);
let mut alter_region_tasks = Vec::with_capacity(leaders.len());
for datanode in leaders {
let requester = self.context.datanode_manager.datanode(&datanode).await;
let regions = find_leader_regions(region_routes, &datanode);
let regions = find_leader_regions(&physical_table_route.region_routes, &datanode);
for region in regions {
let region_id = RegionId::new(table_id, region);
@@ -334,21 +336,31 @@ impl AlterTableProcedure {
Ok(Status::Done)
}
fn lock_key_inner(&self) -> Vec<String> {
fn lock_key_inner(&self) -> Vec<StringKey> {
let mut lock_key = vec![];
if let Some((physical_table_id, physical_table_name)) = self.data.physical_table_info() {
lock_key.push(CatalogLock::Read(&physical_table_name.catalog_name).into());
lock_key.push(
SchemaLock::read(
&physical_table_name.catalog_name,
&physical_table_name.schema_name,
)
.into(),
);
lock_key.push(TableLock::Read(*physical_table_id).into())
}
let table_ref = self.data.table_ref();
let table_key = common_catalog::format_full_table_name(
table_ref.catalog,
table_ref.schema,
table_ref.table,
);
let mut lock_key = vec![table_key];
let table_id = self.data.table_id();
lock_key.push(CatalogLock::Read(table_ref.catalog).into());
lock_key.push(SchemaLock::read(table_ref.catalog, table_ref.schema).into());
lock_key.push(TableLock::Write(table_id).into());
if let Ok(Kind::RenameTable(RenameTable { new_table_name })) = self.alter_kind() {
lock_key.push(common_catalog::format_full_table_name(
table_ref.catalog,
table_ref.schema,
new_table_name,
))
lock_key.push(
TableNameLock::new(table_ref.catalog, table_ref.schema, new_table_name).into(),
)
}
lock_key
@@ -362,8 +374,8 @@ impl Procedure for AlterTableProcedure {
}
async fn execute(&mut self, _ctx: &ProcedureContext) -> ProcedureResult<Status> {
let error_handler = |e| {
if matches!(e, error::Error::RetryLater { .. }) {
let error_handler = |e: Error| {
if e.is_retry_later() {
ProcedureError::retry_later(e)
} else {
ProcedureError::external(e)
@@ -411,11 +423,13 @@ enum AlterTableState {
#[derive(Debug, Serialize, Deserialize)]
pub struct AlterTableData {
cluster_id: u64,
state: AlterTableState,
task: AlterTableTask,
/// Table info value before alteration.
table_info_value: DeserializedValueWithBytes<TableInfoValue>,
cluster_id: u64,
/// Physical table name, if the table to alter is a logical table.
physical_table_info: Option<(TableId, TableName)>,
/// Next column id of the table if the task adds columns to the table.
next_column_id: Option<ColumnId>,
}
@@ -424,6 +438,7 @@ impl AlterTableData {
pub fn new(
task: AlterTableTask,
table_info_value: DeserializedValueWithBytes<TableInfoValue>,
physical_table_info: Option<(TableId, TableName)>,
cluster_id: u64,
next_column_id: Option<ColumnId>,
) -> Self {
@@ -431,6 +446,7 @@ impl AlterTableData {
state: AlterTableState::Prepare,
task,
table_info_value,
physical_table_info,
cluster_id,
next_column_id,
}
@@ -447,6 +463,10 @@ impl AlterTableData {
fn table_info(&self) -> &RawTableInfo {
&self.table_info_value.table_info
}
fn physical_table_info(&self) -> Option<&(TableId, TableName)> {
self.physical_table_info.as_ref()
}
}
/// Creates region proto alter kind from `table_info` and `alter_kind`.

View File

@@ -20,7 +20,6 @@ use api::v1::region::{
};
use api::v1::{ColumnDef, SemanticType};
use async_trait::async_trait;
use common_config::WAL_OPTIONS_KEY;
use common_error::ext::BoxedError;
use common_procedure::error::{
ExternalSnafu, FromJsonSnafu, Result as ProcedureResult, ToJsonSnafu,
@@ -42,12 +41,14 @@ use crate::ddl::DdlContext;
use crate::error::{self, Result, TableRouteNotFoundSnafu};
use crate::key::table_name::TableNameKey;
use crate::key::table_route::TableRouteValue;
use crate::lock_key::TableNameLock;
use crate::metrics;
use crate::region_keeper::OperatingRegionGuard;
use crate::rpc::ddl::CreateTableTask;
use crate::rpc::router::{
find_leader_regions, find_leaders, operating_leader_regions, RegionRoute,
};
use crate::wal::prepare_wal_option;
pub struct CreateTableProcedure {
pub context: DdlContext,
@@ -217,7 +218,7 @@ impl CreateTableProcedure {
.context(TableRouteNotFoundSnafu {
table_id: physical_table_id,
})?;
let region_routes = physical_table_route.region_routes();
let region_routes = physical_table_route.region_routes()?;
let request_builder = self.new_region_request_builder(Some(physical_table_id))?;
@@ -343,13 +344,12 @@ impl Procedure for CreateTableProcedure {
fn lock_key(&self) -> LockKey {
let table_ref = &self.creator.data.table_ref();
let key = common_catalog::format_full_table_name(
LockKey::single(TableNameLock::new(
table_ref.catalog,
table_ref.schema,
table_ref.table,
);
LockKey::single(key)
))
}
}
@@ -455,13 +455,7 @@ impl CreateRequestBuilder {
request.region_id = region_id.as_u64();
request.path = storage_path;
// Stores the encoded wal options into the request options.
region_wal_options
.get(&region_id.region_number())
.and_then(|wal_options| {
request
.options
.insert(WAL_OPTIONS_KEY.to_string(), wal_options.clone())
});
prepare_wal_option(&mut request.options, region_id, region_wal_options);
if let Some(physical_table_id) = self.physical_table_id {
// Logical table has the same region numbers with physical table, and they have a one-to-one mapping.

View File

@@ -41,6 +41,7 @@ use crate::key::table_info::TableInfoValue;
use crate::key::table_name::TableNameKey;
use crate::key::table_route::TableRouteValue;
use crate::key::DeserializedValueWithBytes;
use crate::lock_key::{CatalogLock, SchemaLock, TableLock};
use crate::metrics;
use crate::region_keeper::OperatingRegionGuard;
use crate::rpc::ddl::DropTableTask;
@@ -116,7 +117,7 @@ impl DropTableProcedure {
/// Register dropping regions if doesn't exist.
fn register_dropping_regions(&mut self) -> Result<()> {
let region_routes = self.data.region_routes();
let region_routes = self.data.region_routes()?;
let dropping_regions = operating_leader_regions(region_routes);
@@ -190,7 +191,7 @@ impl DropTableProcedure {
pub async fn on_datanode_drop_regions(&self) -> Result<Status> {
let table_id = self.data.table_id();
let region_routes = &self.data.region_routes();
let region_routes = &self.data.region_routes()?;
let leaders = find_leaders(region_routes);
let mut drop_region_tasks = Vec::with_capacity(leaders.len());
@@ -267,13 +268,14 @@ impl Procedure for DropTableProcedure {
fn lock_key(&self) -> LockKey {
let table_ref = &self.data.table_ref();
let key = common_catalog::format_full_table_name(
table_ref.catalog,
table_ref.schema,
table_ref.table,
);
let table_id = self.data.table_id();
let lock_key = vec![
CatalogLock::Read(table_ref.catalog).into(),
SchemaLock::read(table_ref.catalog, table_ref.schema).into(),
TableLock::Write(table_id).into(),
];
LockKey::single(key)
LockKey::new(lock_key)
}
}
@@ -306,7 +308,7 @@ impl DropTableData {
self.task.table_ref()
}
fn region_routes(&self) -> &Vec<RegionRoute> {
fn region_routes(&self) -> Result<&Vec<RegionRoute>> {
self.table_route_value.region_routes()
}

View File

@@ -0,0 +1,223 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use std::collections::HashMap;
use std::sync::Arc;
use async_trait::async_trait;
use common_catalog::consts::METRIC_ENGINE;
use common_telemetry::{debug, info};
use snafu::{ensure, OptionExt};
use store_api::metric_engine_consts::LOGICAL_TABLE_METADATA_KEY;
use store_api::storage::{RegionId, RegionNumber, TableId};
use crate::ddl::{TableMetadata, TableMetadataAllocatorContext};
use crate::error::{Result, TableNotFoundSnafu, UnsupportedSnafu};
use crate::key::table_name::TableNameKey;
use crate::key::table_route::{LogicalTableRouteValue, PhysicalTableRouteValue, TableRouteValue};
use crate::key::TableMetadataManagerRef;
use crate::peer::Peer;
use crate::rpc::ddl::CreateTableTask;
use crate::rpc::router::{Region, RegionRoute};
use crate::sequence::SequenceRef;
use crate::wal::{allocate_region_wal_options, WalOptionsAllocatorRef};
#[derive(Clone)]
pub struct TableMetadataAllocator {
table_id_sequence: SequenceRef,
wal_options_allocator: WalOptionsAllocatorRef,
table_metadata_manager: TableMetadataManagerRef,
peer_allocator: PeerAllocatorRef,
}
impl TableMetadataAllocator {
pub fn new(
table_id_sequence: SequenceRef,
wal_options_allocator: WalOptionsAllocatorRef,
table_metadata_manager: TableMetadataManagerRef,
) -> Self {
Self::with_peer_allocator(
table_id_sequence,
wal_options_allocator,
table_metadata_manager,
Arc::new(NoopPeerAllocator),
)
}
pub fn with_peer_allocator(
table_id_sequence: SequenceRef,
wal_options_allocator: WalOptionsAllocatorRef,
table_metadata_manager: TableMetadataManagerRef,
peer_allocator: PeerAllocatorRef,
) -> Self {
Self {
table_id_sequence,
wal_options_allocator,
table_metadata_manager,
peer_allocator,
}
}
async fn allocate_table_id(&self, task: &CreateTableTask) -> Result<TableId> {
let table_id = if let Some(table_id) = &task.create_table.table_id {
let table_id = table_id.id;
ensure!(
!self
.table_id_sequence
.min_max()
.await
.contains(&(table_id as u64)),
UnsupportedSnafu {
operation: format!(
"create table by id {} that is reserved in this node",
table_id
)
}
);
info!(
"Received explicitly allocated table id {}, will use it directly.",
table_id
);
table_id
} else {
self.table_id_sequence.next().await? as TableId
};
Ok(table_id)
}
fn create_wal_options(
&self,
table_route: &TableRouteValue,
) -> Result<HashMap<RegionNumber, String>> {
match table_route {
TableRouteValue::Physical(x) => {
let region_numbers = x
.region_routes
.iter()
.map(|route| route.region.id.region_number())
.collect();
allocate_region_wal_options(region_numbers, &self.wal_options_allocator)
}
TableRouteValue::Logical(_) => Ok(HashMap::new()),
}
}
async fn create_table_route(
&self,
ctx: &TableMetadataAllocatorContext,
table_id: TableId,
task: &CreateTableTask,
) -> Result<TableRouteValue> {
let regions = task.partitions.len();
let table_route = if task.create_table.engine == METRIC_ENGINE
&& let Some(physical_table_name) = task
.create_table
.table_options
.get(LOGICAL_TABLE_METADATA_KEY)
{
let physical_table_id = self
.table_metadata_manager
.table_name_manager()
.get(TableNameKey::new(
&task.create_table.catalog_name,
&task.create_table.schema_name,
physical_table_name,
))
.await?
.context(TableNotFoundSnafu {
table_name: physical_table_name,
})?
.table_id();
let region_ids = (0..regions)
.map(|i| RegionId::new(table_id, i as RegionNumber))
.collect();
TableRouteValue::Logical(LogicalTableRouteValue::new(physical_table_id, region_ids))
} else {
let peers = self.peer_allocator.alloc(ctx, regions).await?;
let region_routes = task
.partitions
.iter()
.enumerate()
.map(|(i, partition)| {
let region = Region {
id: RegionId::new(table_id, i as u32),
partition: Some(partition.clone().into()),
..Default::default()
};
let peer = peers[i % peers.len()].clone();
RegionRoute {
region,
leader_peer: Some(peer),
..Default::default()
}
})
.collect::<Vec<_>>();
TableRouteValue::Physical(PhysicalTableRouteValue::new(region_routes))
};
Ok(table_route)
}
pub async fn create(
&self,
ctx: &TableMetadataAllocatorContext,
task: &CreateTableTask,
) -> Result<TableMetadata> {
let table_id = self.allocate_table_id(task).await?;
let table_route = self.create_table_route(ctx, table_id, task).await?;
let region_wal_options = self.create_wal_options(&table_route)?;
debug!(
"Allocated region wal options {:?} for table {}",
region_wal_options, table_id
);
Ok(TableMetadata {
table_id,
table_route,
region_wal_options,
})
}
}
pub type PeerAllocatorRef = Arc<dyn PeerAllocator>;
/// [PeerAllocator] allocates [Peer]s for creating regions.
#[async_trait]
pub trait PeerAllocator: Send + Sync {
/// Allocates `regions` size [Peer]s.
async fn alloc(&self, ctx: &TableMetadataAllocatorContext, regions: usize)
-> Result<Vec<Peer>>;
}
struct NoopPeerAllocator;
#[async_trait]
impl PeerAllocator for NoopPeerAllocator {
async fn alloc(
&self,
_ctx: &TableMetadataAllocatorContext,
regions: usize,
) -> Result<Vec<Peer>> {
Ok(vec![Peer::default(); regions])
}
}

View File

@@ -37,6 +37,7 @@ use crate::error::{Result, TableNotFoundSnafu};
use crate::key::table_info::TableInfoValue;
use crate::key::table_name::TableNameKey;
use crate::key::DeserializedValueWithBytes;
use crate::lock_key::{CatalogLock, SchemaLock, TableLock};
use crate::metrics;
use crate::rpc::ddl::TruncateTableTask;
use crate::rpc::router::{find_leader_regions, find_leaders, RegionRoute};
@@ -75,13 +76,14 @@ impl Procedure for TruncateTableProcedure {
fn lock_key(&self) -> LockKey {
let table_ref = &self.data.table_ref();
let key = common_catalog::format_full_table_name(
table_ref.catalog,
table_ref.schema,
table_ref.table,
);
let table_id = self.data.table_id();
let lock_key = vec![
CatalogLock::Read(table_ref.catalog).into(),
SchemaLock::read(table_ref.catalog, table_ref.schema).into(),
TableLock::Write(table_id).into(),
];
LockKey::single(key)
LockKey::new(lock_key)
}
}

View File

@@ -36,7 +36,7 @@ pub fn handle_operate_region_error(datanode: Peer) -> impl FnOnce(crate::error::
}
pub fn handle_retry_error(e: Error) -> ProcedureError {
if matches!(e, error::Error::RetryLater { .. }) {
if e.is_retry_later() {
ProcedureError::retry_later(e)
} else {
ProcedureError::external(e)

View File

@@ -19,17 +19,17 @@ use common_procedure::{watcher, ProcedureId, ProcedureManagerRef, ProcedureWithI
use common_telemetry::tracing_context::{FutureExt, TracingContext};
use common_telemetry::{info, tracing};
use snafu::{OptionExt, ResultExt};
use store_api::storage::RegionNumber;
use store_api::storage::{RegionNumber, TableId};
use crate::cache_invalidator::CacheInvalidatorRef;
use crate::datanode_manager::DatanodeManagerRef;
use crate::ddl::alter_table::AlterTableProcedure;
use crate::ddl::create_table::CreateTableProcedure;
use crate::ddl::drop_table::DropTableProcedure;
use crate::ddl::table_meta::TableMetadataAllocator;
use crate::ddl::truncate_table::TruncateTableProcedure;
use crate::ddl::{
DdlContext, DdlTaskExecutor, ExecutorContext, TableMetadata, TableMetadataAllocatorContext,
TableMetadataAllocatorRef,
};
use crate::error::{
self, RegisterProcedureLoaderSnafu, Result, SubmitProcedureSnafu, TableNotFoundSnafu,
@@ -46,6 +46,8 @@ use crate::rpc::ddl::{
TruncateTableTask,
};
use crate::rpc::router::RegionRoute;
use crate::table_name::TableName;
pub type DdlManagerRef = Arc<DdlManager>;
/// The [DdlManager] provides the ability to execute Ddl.
@@ -54,7 +56,7 @@ pub struct DdlManager {
datanode_manager: DatanodeManagerRef,
cache_invalidator: CacheInvalidatorRef,
table_metadata_manager: TableMetadataManagerRef,
table_metadata_allocator: TableMetadataAllocatorRef,
table_metadata_allocator: TableMetadataAllocator,
memory_region_keeper: MemoryRegionKeeperRef,
}
@@ -65,7 +67,7 @@ impl DdlManager {
datanode_clients: DatanodeManagerRef,
cache_invalidator: CacheInvalidatorRef,
table_metadata_manager: TableMetadataManagerRef,
table_metadata_allocator: TableMetadataAllocatorRef,
table_metadata_allocator: TableMetadataAllocator,
memory_region_keeper: MemoryRegionKeeperRef,
) -> Result<Self> {
let manager = Self {
@@ -160,11 +162,17 @@ impl DdlManager {
cluster_id: u64,
alter_table_task: AlterTableTask,
table_info_value: DeserializedValueWithBytes<TableInfoValue>,
physical_table_info: Option<(TableId, TableName)>,
) -> Result<ProcedureId> {
let context = self.create_context();
let procedure =
AlterTableProcedure::new(cluster_id, alter_table_task, table_info_value, context)?;
let procedure = AlterTableProcedure::new(
cluster_id,
alter_table_task,
table_info_value,
physical_table_info,
context,
)?;
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
@@ -278,7 +286,7 @@ async fn handle_truncate_table_task(
let table_route_value =
table_route_value.context(error::TableRouteNotFoundSnafu { table_id })?;
let table_route = table_route_value.into_inner().region_routes().clone();
let table_route = table_route_value.into_inner().region_routes()?.clone();
let id = ddl_manager
.submit_truncate_table_task(
@@ -327,8 +335,41 @@ async fn handle_alter_table_task(
table_name: table_ref.to_string(),
})?;
let physical_table_id = ddl_manager
.table_metadata_manager()
.table_route_manager()
.get_physical_table_id(table_id)
.await?;
let physical_table_info = if physical_table_id == table_id {
None
} else {
let physical_table_info = &ddl_manager
.table_metadata_manager()
.table_info_manager()
.get(physical_table_id)
.await?
.with_context(|| error::TableInfoNotFoundSnafu {
table_name: table_ref.to_string(),
})?
.table_info;
Some((
physical_table_id,
TableName {
catalog_name: physical_table_info.catalog_name.clone(),
schema_name: physical_table_info.schema_name.clone(),
table_name: physical_table_info.name.clone(),
},
))
};
let id = ddl_manager
.submit_alter_table_task(cluster_id, alter_table_task, table_info_value)
.submit_alter_table_task(
cluster_id,
alter_table_task,
table_info_value,
physical_table_info,
)
.await?;
info!("Table: {table_id} is altered via procedure_id {id:?}");
@@ -348,15 +389,21 @@ async fn handle_drop_table_task(
let table_metadata_manager = &ddl_manager.table_metadata_manager();
let table_ref = drop_table_task.table_ref();
let (table_info_value, table_route_value) =
table_metadata_manager.get_full_table_info(table_id).await?;
let table_info_value = table_metadata_manager
.table_info_manager()
.get(table_id)
.await?;
let (_, table_route_value) = table_metadata_manager
.table_route_manager()
.get_physical_table_route(table_id)
.await?;
let table_info_value = table_info_value.with_context(|| error::TableInfoNotFoundSnafu {
table_name: table_ref.to_string(),
})?;
let table_route_value =
table_route_value.context(error::TableRouteNotFoundSnafu { table_id })?;
DeserializedValueWithBytes::from_inner(TableRouteValue::Physical(table_route_value));
let id = ddl_manager
.submit_drop_table_task(
@@ -461,15 +508,15 @@ mod tests {
use crate::ddl::alter_table::AlterTableProcedure;
use crate::ddl::create_table::CreateTableProcedure;
use crate::ddl::drop_table::DropTableProcedure;
use crate::ddl::table_meta::TableMetadataAllocator;
use crate::ddl::truncate_table::TruncateTableProcedure;
use crate::ddl::{TableMetadata, TableMetadataAllocator, TableMetadataAllocatorContext};
use crate::error::Result;
use crate::key::TableMetadataManager;
use crate::kv_backend::memory::MemoryKvBackend;
use crate::peer::Peer;
use crate::region_keeper::MemoryRegionKeeper;
use crate::rpc::ddl::CreateTableTask;
use crate::sequence::SequenceBuilder;
use crate::state_store::KvStateStore;
use crate::wal::WalOptionsAllocator;
/// A dummy implemented [DatanodeManager].
pub struct DummyDatanodeManager;
@@ -481,26 +528,12 @@ mod tests {
}
}
/// A dummy implemented [TableMetadataAllocator].
pub struct DummyTableMetadataAllocator;
#[async_trait::async_trait]
impl TableMetadataAllocator for DummyTableMetadataAllocator {
async fn create(
&self,
_ctx: &TableMetadataAllocatorContext,
_task: &CreateTableTask,
) -> Result<TableMetadata> {
unimplemented!()
}
}
#[test]
fn test_try_new() {
let kv_backend = Arc::new(MemoryKvBackend::new());
let table_metadata_manager = Arc::new(TableMetadataManager::new(kv_backend.clone()));
let state_store = Arc::new(KvStateStore::new(kv_backend));
let state_store = Arc::new(KvStateStore::new(kv_backend.clone()));
let procedure_manager = Arc::new(LocalManager::new(Default::default(), state_store));
let _ = DdlManager::try_new(
@@ -508,7 +541,11 @@ mod tests {
Arc::new(DummyDatanodeManager),
Arc::new(DummyCacheInvalidator),
table_metadata_manager,
Arc::new(DummyTableMetadataAllocator),
TableMetadataAllocator::new(
Arc::new(SequenceBuilder::new("test", kv_backend.clone()).build()),
Arc::new(WalOptionsAllocator::default()),
Arc::new(TableMetadataManager::new(kv_backend)),
),
Arc::new(MemoryRegionKeeper::default()),
);

View File

@@ -321,6 +321,27 @@ pub enum Error {
error: rskafka::client::error::Error,
},
#[snafu(display(
"Failed to build a Kafka partition client, topic: {}, partition: {}",
topic,
partition
))]
BuildKafkaPartitionClient {
topic: String,
partition: i32,
location: Location,
#[snafu(source)]
error: rskafka::client::error::Error,
},
#[snafu(display("Failed to produce records to Kafka, topic: {}", topic))]
ProduceRecord {
topic: String,
location: Location,
#[snafu(source)]
error: rskafka::client::error::Error,
},
#[snafu(display("Failed to create a Kafka wal topic"))]
CreateKafkaWalTopic {
location: Location,
@@ -330,6 +351,9 @@ pub enum Error {
#[snafu(display("The topic pool is empty"))]
EmptyTopicPool { location: Location },
#[snafu(display("Unexpected table route type: {}", err_msg))]
UnexpectedLogicalRouteTable { location: Location, err_msg: String },
}
pub type Result<T> = std::result::Result<T, Error>;
@@ -368,8 +392,11 @@ impl ErrorExt for Error {
| EncodeWalOptions { .. }
| BuildKafkaClient { .. }
| BuildKafkaCtrlClient { .. }
| BuildKafkaPartitionClient { .. }
| ProduceRecord { .. }
| CreateKafkaWalTopic { .. }
| EmptyTopicPool { .. } => StatusCode::Unexpected,
| EmptyTopicPool { .. }
| UnexpectedLogicalRouteTable { .. } => StatusCode::Unexpected,
SendMessage { .. }
| GetKvCache { .. }

View File

@@ -92,13 +92,15 @@ impl Display for OpenRegion {
}
}
#[serde_with::serde_as]
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)]
pub struct OpenRegion {
pub region_ident: RegionIdent,
pub region_storage_path: String,
pub region_options: HashMap<String, String>,
#[serde(default)]
pub region_wal_options: HashMap<String, String>,
#[serde_as(as = "HashMap<serde_with::DisplayFromStr, _>")]
pub region_wal_options: HashMap<RegionNumber, String>,
#[serde(default)]
pub skip_wal_replay: bool,
}
@@ -108,7 +110,7 @@ impl OpenRegion {
region_ident: RegionIdent,
path: &str,
region_options: HashMap<String, String>,
region_wal_options: HashMap<String, String>,
region_wal_options: HashMap<RegionNumber, String>,
skip_wal_replay: bool,
) -> Self {
Self {

View File

@@ -354,6 +354,7 @@ impl TableMetadataManager {
&self.kv_backend
}
// TODO(ruihang): deprecate this
pub async fn get_full_table_info(
&self,
table_id: TableId,
@@ -427,7 +428,7 @@ impl TableMetadataManager {
&region_storage_path,
region_options,
region_wal_options,
region_distribution(&x.region_routes)?,
region_distribution(&x.region_routes),
)?;
txn = txn.merge(create_datanode_table_txn);
}
@@ -483,7 +484,7 @@ impl TableMetadataManager {
.build_delete_txn(table_id, table_info_value)?;
// Deletes datanode table key value pairs.
let distribution = region_distribution(table_route_value.region_routes())?;
let distribution = region_distribution(table_route_value.region_routes()?);
let delete_datanode_txn = self
.datanode_table_manager()
.build_delete_txn(table_id, distribution)?;
@@ -604,12 +605,12 @@ impl TableMetadataManager {
current_table_route_value: &DeserializedValueWithBytes<TableRouteValue>,
new_region_routes: Vec<RegionRoute>,
new_region_options: &HashMap<String, String>,
new_region_wal_options: &HashMap<String, String>,
new_region_wal_options: &HashMap<RegionNumber, String>,
) -> Result<()> {
// Updates the datanode table key value pairs.
let current_region_distribution =
region_distribution(current_table_route_value.region_routes())?;
let new_region_distribution = region_distribution(&new_region_routes)?;
region_distribution(current_table_route_value.region_routes()?);
let new_region_distribution = region_distribution(&new_region_routes);
let update_datanode_table_txn = self.datanode_table_manager().build_update_txn(
table_id,
@@ -621,7 +622,7 @@ impl TableMetadataManager {
)?;
// Updates the table_route.
let new_table_route_value = current_table_route_value.update(new_region_routes);
let new_table_route_value = current_table_route_value.update(new_region_routes)?;
let (update_table_route_txn, on_update_table_route_failure) = self
.table_route_manager()
@@ -656,7 +657,7 @@ impl TableMetadataManager {
where
F: Fn(&RegionRoute) -> Option<Option<RegionStatus>>,
{
let mut new_region_routes = current_table_route_value.region_routes().clone();
let mut new_region_routes = current_table_route_value.region_routes()?.clone();
let mut updated = 0;
for route in &mut new_region_routes {
@@ -673,7 +674,7 @@ impl TableMetadataManager {
}
// Updates the table_route.
let new_table_route_value = current_table_route_value.update(new_region_routes);
let new_table_route_value = current_table_route_value.update(new_region_routes)?;
let (update_table_route_txn, on_update_table_route_failure) = self
.table_route_manager()
@@ -897,7 +898,11 @@ mod tests {
table_info
);
assert_eq!(
remote_table_route.unwrap().into_inner().region_routes(),
remote_table_route
.unwrap()
.into_inner()
.region_routes()
.unwrap(),
region_routes
);
}
@@ -978,7 +983,7 @@ mod tests {
.unwrap()
.unwrap()
.into_inner();
assert_eq!(removed_table_route.region_routes(), region_routes);
assert_eq!(removed_table_route.region_routes().unwrap(), region_routes);
}
#[tokio::test]
@@ -1173,11 +1178,11 @@ mod tests {
.unwrap();
assert_eq!(
updated_route_value.region_routes()[0].leader_status,
updated_route_value.region_routes().unwrap()[0].leader_status,
Some(RegionStatus::Downgraded)
);
assert_eq!(
updated_route_value.region_routes()[1].leader_status,
updated_route_value.region_routes().unwrap()[1].leader_status,
Some(RegionStatus::Downgraded)
);
}
@@ -1187,7 +1192,7 @@ mod tests {
table_id: u32,
region_routes: &[RegionRoute],
) {
let region_distribution = region_distribution(region_routes).unwrap();
let region_distribution = region_distribution(region_routes);
for (datanode, regions) in region_distribution {
let got = table_metadata_manager
.datanode_table_manager()
@@ -1271,7 +1276,8 @@ mod tests {
let current_table_route_value = DeserializedValueWithBytes::from_inner(
current_table_route_value
.inner
.update(new_region_routes.clone()),
.update(new_region_routes.clone())
.unwrap(),
);
let new_region_routes = vec![new_region_route(2, 4), new_region_route(5, 5)];
// it should be ok.
@@ -1295,13 +1301,16 @@ mod tests {
// if the current_table_route_value is wrong, it should return an error.
// The ABA problem.
let wrong_table_route_value =
DeserializedValueWithBytes::from_inner(current_table_route_value.update(vec![
new_region_route(1, 1),
new_region_route(2, 2),
new_region_route(3, 3),
new_region_route(4, 4),
]));
let wrong_table_route_value = DeserializedValueWithBytes::from_inner(
current_table_route_value
.update(vec![
new_region_route(1, 1),
new_region_route(2, 2),
new_region_route(3, 3),
new_region_route(4, 4),
])
.unwrap(),
);
assert!(table_metadata_manager
.update_table_route(
table_id,

View File

@@ -34,6 +34,7 @@ use crate::rpc::store::RangeRequest;
use crate::rpc::KeyValue;
use crate::DatanodeId;
#[serde_with::serde_as]
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Default)]
/// RegionInfo
/// For compatible reason, DON'T modify the field name.
@@ -48,14 +49,15 @@ pub struct RegionInfo {
#[serde(default)]
pub region_options: HashMap<String, String>,
/// The per-region wal options.
/// Key: region number (in string representation). Value: the encoded wal options of the region.
/// Key: region number. Value: the encoded wal options of the region.
#[serde(default)]
pub region_wal_options: HashMap<String, String>,
#[serde_as(as = "HashMap<serde_with::DisplayFromStr, _>")]
pub region_wal_options: HashMap<RegionNumber, String>,
}
pub struct DatanodeTableKey {
datanode_id: DatanodeId,
table_id: TableId,
pub datanode_id: DatanodeId,
pub table_id: TableId,
}
impl DatanodeTableKey {
@@ -176,15 +178,6 @@ impl DatanodeTableManager {
let txns = distribution
.into_iter()
.map(|(datanode_id, regions)| {
let filtered_region_wal_options = regions
.iter()
.filter_map(|region_number| {
region_wal_options
.get(region_number)
.map(|wal_options| (region_number.to_string(), wal_options.clone()))
})
.collect();
let key = DatanodeTableKey::new(datanode_id, table_id);
let val = DatanodeTableValue::new(
table_id,
@@ -193,7 +186,9 @@ impl DatanodeTableManager {
engine: engine.to_string(),
region_storage_path: region_storage_path.to_string(),
region_options: region_options.clone(),
region_wal_options: filtered_region_wal_options,
// FIXME(weny): Before we store all region wal options into table metadata or somewhere,
// We must store all region wal options.
region_wal_options: region_wal_options.clone(),
},
);
@@ -214,7 +209,7 @@ impl DatanodeTableManager {
current_region_distribution: RegionDistribution,
new_region_distribution: RegionDistribution,
new_region_options: &HashMap<String, String>,
new_region_wal_options: &HashMap<String, String>,
new_region_wal_options: &HashMap<RegionNumber, String>,
) -> Result<Txn> {
let mut opts = Vec::new();
@@ -241,7 +236,15 @@ impl DatanodeTableManager {
if need_update {
let key = DatanodeTableKey::new(datanode, table_id);
let raw_key = key.as_raw_key();
let val = DatanodeTableValue::new(table_id, regions, region_info.clone())
// FIXME(weny): add unit tests.
let mut new_region_info = region_info.clone();
if need_update_options {
new_region_info.region_options = new_region_options.clone();
}
if need_update_wal_options {
new_region_info.region_wal_options = new_region_wal_options.clone();
}
let val = DatanodeTableValue::new(table_id, regions, new_region_info)
.try_as_raw_value()?;
opts.push(TxnOp::Put(raw_key, val));
}
@@ -306,6 +309,61 @@ mod tests {
assert!(parsed.is_ok());
}
#[derive(Debug, Serialize, Deserialize, PartialEq)]
struct StringHashMap {
inner: HashMap<String, String>,
}
#[serde_with::serde_as]
#[derive(Debug, Serialize, Deserialize, PartialEq)]
struct IntegerHashMap {
#[serde_as(as = "HashMap<serde_with::DisplayFromStr, _>")]
inner: HashMap<u32, String>,
}
#[test]
fn test_serde_with_integer_hash_map() {
let map = StringHashMap {
inner: HashMap::from([
("1".to_string(), "aaa".to_string()),
("2".to_string(), "bbb".to_string()),
("3".to_string(), "ccc".to_string()),
]),
};
let encoded = serde_json::to_string(&map).unwrap();
let decoded: IntegerHashMap = serde_json::from_str(&encoded).unwrap();
assert_eq!(
IntegerHashMap {
inner: HashMap::from([
(1, "aaa".to_string()),
(2, "bbb".to_string()),
(3, "ccc".to_string()),
]),
},
decoded
);
let map = IntegerHashMap {
inner: HashMap::from([
(1, "aaa".to_string()),
(2, "bbb".to_string()),
(3, "ccc".to_string()),
]),
};
let encoded = serde_json::to_string(&map).unwrap();
let decoded: StringHashMap = serde_json::from_str(&encoded).unwrap();
assert_eq!(
StringHashMap {
inner: HashMap::from([
("1".to_string(), "aaa".to_string()),
("2".to_string(), "bbb".to_string()),
("3".to_string(), "ccc".to_string()),
]),
},
decoded
);
}
// This test intends to ensure both the `serde_json::to_string` + `serde_json::from_str`
// and `serde_json::to_vec` + `serde_json::from_slice` work for `DatanodeTableValue`.
// Warning: if the key of `region_wal_options` is of type non-String, this test would fail.
@@ -320,9 +378,9 @@ mod tests {
("c".to_string(), "cc".to_string()),
]),
region_wal_options: HashMap::from([
("1".to_string(), "aaa".to_string()),
("2".to_string(), "bbb".to_string()),
("3".to_string(), "ccc".to_string()),
(1, "aaa".to_string()),
(2, "bbb".to_string()),
(3, "ccc".to_string()),
]),
};
let table_value = DatanodeTableValue {

View File

@@ -16,12 +16,14 @@ use std::collections::HashMap;
use std::fmt::Display;
use serde::{Deserialize, Serialize};
use snafu::ResultExt;
use snafu::{ensure, OptionExt, ResultExt};
use store_api::storage::{RegionId, RegionNumber};
use table::metadata::TableId;
use super::{DeserializedValueWithBytes, TableMetaValue};
use crate::error::{Result, SerdeJsonSnafu};
use crate::error::{
Result, SerdeJsonSnafu, TableRouteNotFoundSnafu, UnexpectedLogicalRouteTableSnafu,
};
use crate::key::{to_removed_key, RegionDistribution, TableMetaKey, TABLE_ROUTE_PREFIX};
use crate::kv_backend::txn::{Compare, CompareOp, Txn, TxnOp, TxnOpResponse};
use crate::kv_backend::KvBackendRef;
@@ -53,7 +55,8 @@ pub struct PhysicalTableRouteValue {
#[derive(Debug, PartialEq, Serialize, Deserialize, Clone)]
pub struct LogicalTableRouteValue {
// TODO(LFC): Add table route for MetricsEngine table.
physical_table_id: TableId,
region_ids: Vec<RegionId>,
}
impl TableRouteValue {
@@ -62,29 +65,50 @@ impl TableRouteValue {
}
/// Returns a new version [TableRouteValue] with `region_routes`.
pub fn update(&self, region_routes: Vec<RegionRoute>) -> Self {
pub fn update(&self, region_routes: Vec<RegionRoute>) -> Result<Self> {
ensure!(
self.is_physical(),
UnexpectedLogicalRouteTableSnafu {
err_msg: format!("{self:?} is a non-physical TableRouteValue."),
}
);
let version = self.physical_table_route().version;
Self::Physical(PhysicalTableRouteValue {
Ok(Self::Physical(PhysicalTableRouteValue {
region_routes,
version: version + 1,
})
}))
}
/// Returns the version.
///
/// For test purpose.
#[cfg(any(test, feature = "testing"))]
pub fn version(&self) -> u64 {
self.physical_table_route().version
pub fn version(&self) -> Result<u64> {
ensure!(
self.is_physical(),
UnexpectedLogicalRouteTableSnafu {
err_msg: format!("{self:?} is a non-physical TableRouteValue."),
}
);
Ok(self.physical_table_route().version)
}
/// Returns the corresponding [RegionRoute].
pub fn region_route(&self, region_id: RegionId) -> Option<RegionRoute> {
self.physical_table_route()
/// Returns the corresponding [RegionRoute], returns `None` if it's the specific region is not found.
///
/// Note: It throws an error if it's a logical table
pub fn region_route(&self, region_id: RegionId) -> Result<Option<RegionRoute>> {
ensure!(
self.is_physical(),
UnexpectedLogicalRouteTableSnafu {
err_msg: format!("{self:?} is a non-physical TableRouteValue."),
}
);
Ok(self
.physical_table_route()
.region_routes
.iter()
.find(|route| route.region.id == region_id)
.cloned()
.cloned())
}
/// Returns true if it's [TableRouteValue::Physical].
@@ -93,11 +117,14 @@ impl TableRouteValue {
}
/// Gets the [RegionRoute]s of this [TableRouteValue::Physical].
///
/// # Panics
/// The route type is not the [TableRouteValue::Physical].
pub fn region_routes(&self) -> &Vec<RegionRoute> {
&self.physical_table_route().region_routes
pub fn region_routes(&self) -> Result<&Vec<RegionRoute>> {
ensure!(
self.is_physical(),
UnexpectedLogicalRouteTableSnafu {
err_msg: format!("{self:?} is a non-physical TableRouteValue."),
}
);
Ok(&self.physical_table_route().region_routes)
}
fn physical_table_route(&self) -> &PhysicalTableRouteValue {
@@ -152,12 +179,19 @@ impl PhysicalTableRouteValue {
}
impl LogicalTableRouteValue {
pub fn physical_table_id(&self) -> TableId {
todo!()
pub fn new(physical_table_id: TableId, region_ids: Vec<RegionId>) -> Self {
Self {
physical_table_id,
region_ids,
}
}
pub fn region_ids(&self) -> Vec<RegionId> {
todo!()
pub fn physical_table_id(&self) -> TableId {
self.physical_table_id
}
pub fn region_ids(&self) -> &Vec<RegionId> {
&self.region_ids
}
}
@@ -302,6 +336,54 @@ impl TableRouteManager {
.transpose()
}
pub async fn get_physical_table_id(
&self,
logical_or_physical_table_id: TableId,
) -> Result<TableId> {
let table_route = self
.get(logical_or_physical_table_id)
.await?
.context(TableRouteNotFoundSnafu {
table_id: logical_or_physical_table_id,
})?
.into_inner();
match table_route {
TableRouteValue::Physical(_) => Ok(logical_or_physical_table_id),
TableRouteValue::Logical(x) => Ok(x.physical_table_id()),
}
}
pub async fn get_physical_table_route(
&self,
logical_or_physical_table_id: TableId,
) -> Result<(TableId, PhysicalTableRouteValue)> {
let table_route = self
.get(logical_or_physical_table_id)
.await?
.context(TableRouteNotFoundSnafu {
table_id: logical_or_physical_table_id,
})?
.into_inner();
match table_route {
TableRouteValue::Physical(x) => Ok((logical_or_physical_table_id, x)),
TableRouteValue::Logical(x) => {
let physical_table_id = x.physical_table_id();
let physical_table_route =
self.get(physical_table_id)
.await?
.context(TableRouteNotFoundSnafu {
table_id: physical_table_id,
})?;
Ok((
physical_table_id,
physical_table_route.physical_table_route().clone(),
))
}
}
}
/// It may return a subset of the `table_ids`.
pub async fn batch_get(
&self,
@@ -354,7 +436,7 @@ impl TableRouteManager {
) -> Result<Option<RegionDistribution>> {
self.get(table_id)
.await?
.map(|table_route| region_distribution(table_route.region_routes()))
.map(|table_route| Ok(region_distribution(table_route.region_routes()?)))
.transpose()
}
}

View File

@@ -114,6 +114,7 @@ where
Ok(!resp.kvs.is_empty())
}
/// Returns previous key-value pair if `prev_kv` is `true`.
async fn delete(&self, key: &[u8], prev_kv: bool) -> Result<Option<KeyValue>, Self::Error> {
let mut req = DeleteRangeRequest::new().with_key(key.to_vec());
if prev_kv {

View File

@@ -15,6 +15,7 @@
#![feature(assert_matches)]
#![feature(btree_extract_if)]
#![feature(async_closure)]
#![feature(let_chains)]
pub mod cache_invalidator;
pub mod datanode_manager;
@@ -26,6 +27,7 @@ pub mod heartbeat;
pub mod instruction;
pub mod key;
pub mod kv_backend;
pub mod lock_key;
pub mod metrics;
pub mod peer;
pub mod range_stream;
@@ -35,7 +37,6 @@ pub mod sequence;
pub mod state_store;
pub mod table_name;
pub mod util;
#[allow(unused)]
pub mod wal;
pub type ClusterId = u64;

View File

@@ -0,0 +1,235 @@
// 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::fmt::Display;
use common_catalog::{format_full_table_name, format_schema_name};
use common_procedure::StringKey;
use store_api::storage::{RegionId, TableId};
const CATALOG_LOCK_PREFIX: &str = "__catalog_lock";
const SCHEMA_LOCK_PREFIX: &str = "__schema_lock";
const TABLE_LOCK_PREFIX: &str = "__table_lock";
const TABLE_NAME_LOCK_PREFIX: &str = "__table_name_lock";
const REGION_LOCK_PREFIX: &str = "__region_lock";
/// [CatalogLock] acquires the lock on the tenant level.
pub enum CatalogLock<'a> {
Read(&'a str),
Write(&'a str),
}
impl<'a> Display for CatalogLock<'a> {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
let key = match self {
CatalogLock::Read(s) => s,
CatalogLock::Write(s) => s,
};
write!(f, "{}/{}", CATALOG_LOCK_PREFIX, key)
}
}
impl<'a> From<CatalogLock<'a>> for StringKey {
fn from(value: CatalogLock) -> Self {
match value {
CatalogLock::Write(_) => StringKey::Exclusive(value.to_string()),
CatalogLock::Read(_) => StringKey::Share(value.to_string()),
}
}
}
/// [SchemaLock] acquires the lock on the database level.
pub enum SchemaLock {
Read(String),
Write(String),
}
impl SchemaLock {
pub fn read(catalog: &str, schema: &str) -> Self {
Self::Read(format_schema_name(catalog, schema))
}
pub fn write(catalog: &str, schema: &str) -> Self {
Self::Write(format_schema_name(catalog, schema))
}
}
impl Display for SchemaLock {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
let key = match self {
SchemaLock::Read(s) => s,
SchemaLock::Write(s) => s,
};
write!(f, "{}/{}", SCHEMA_LOCK_PREFIX, key)
}
}
impl From<SchemaLock> for StringKey {
fn from(value: SchemaLock) -> Self {
match value {
SchemaLock::Write(_) => StringKey::Exclusive(value.to_string()),
SchemaLock::Read(_) => StringKey::Share(value.to_string()),
}
}
}
/// [TableNameLock] prevents any procedures trying to create a table named it.
pub enum TableNameLock {
Write(String),
}
impl TableNameLock {
pub fn new(catalog: &str, schema: &str, table: &str) -> Self {
Self::Write(format_full_table_name(catalog, schema, table))
}
}
impl Display for TableNameLock {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
let TableNameLock::Write(name) = self;
write!(f, "{}/{}", TABLE_NAME_LOCK_PREFIX, name)
}
}
impl From<TableNameLock> for StringKey {
fn from(value: TableNameLock) -> Self {
match value {
TableNameLock::Write(_) => StringKey::Exclusive(value.to_string()),
}
}
}
/// [TableLock] acquires the lock on the table level.
///
/// Note: Allows to read/modify the corresponding table's [TableInfoValue](crate::key::table_info::TableInfoValue),
/// [TableRouteValue](crate::key::table_route::TableRouteValue), [TableDatanodeValue](crate::key::datanode_table::DatanodeTableValue).
pub enum TableLock {
Read(TableId),
Write(TableId),
}
impl Display for TableLock {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
let key = match self {
TableLock::Read(s) => s,
TableLock::Write(s) => s,
};
write!(f, "{}/{}", TABLE_LOCK_PREFIX, key)
}
}
impl From<TableLock> for StringKey {
fn from(value: TableLock) -> Self {
match value {
TableLock::Write(_) => StringKey::Exclusive(value.to_string()),
TableLock::Read(_) => StringKey::Share(value.to_string()),
}
}
}
/// [RegionLock] acquires the lock on the region level.
///
/// Note:
/// - Allows modification the corresponding region's [TableRouteValue](crate::key::table_route::TableRouteValue),
/// [TableDatanodeValue](crate::key::datanode_table::DatanodeTableValue) even if
/// it acquires the [RegionLock::Write] only without acquiring the [TableLock::Write].
///
/// - Should acquire [TableLock] of the table at same procedure.
///
/// TODO(weny): we should consider separating TableRouteValue into finer keys.
pub enum RegionLock {
Read(RegionId),
Write(RegionId),
}
impl Display for RegionLock {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
let key = match self {
RegionLock::Read(s) => s.as_u64(),
RegionLock::Write(s) => s.as_u64(),
};
write!(f, "{}/{}", REGION_LOCK_PREFIX, key)
}
}
impl From<RegionLock> for StringKey {
fn from(value: RegionLock) -> Self {
match value {
RegionLock::Write(_) => StringKey::Exclusive(value.to_string()),
RegionLock::Read(_) => StringKey::Share(value.to_string()),
}
}
}
#[cfg(test)]
mod tests {
use common_procedure::StringKey;
use crate::lock_key::*;
#[test]
fn test_lock_key() {
// The catalog lock
let string_key: StringKey = CatalogLock::Read("foo").into();
assert_eq!(
string_key,
StringKey::Share(format!("{}/{}", CATALOG_LOCK_PREFIX, "foo"))
);
let string_key: StringKey = CatalogLock::Write("foo").into();
assert_eq!(
string_key,
StringKey::Exclusive(format!("{}/{}", CATALOG_LOCK_PREFIX, "foo"))
);
// The schema lock
let string_key: StringKey = SchemaLock::read("foo", "bar").into();
assert_eq!(
string_key,
StringKey::Share(format!("{}/{}", SCHEMA_LOCK_PREFIX, "foo.bar"))
);
let string_key: StringKey = SchemaLock::write("foo", "bar").into();
assert_eq!(
string_key,
StringKey::Exclusive(format!("{}/{}", SCHEMA_LOCK_PREFIX, "foo.bar"))
);
// The table lock
let string_key: StringKey = TableLock::Read(1024).into();
assert_eq!(
string_key,
StringKey::Share(format!("{}/{}", TABLE_LOCK_PREFIX, 1024))
);
let string_key: StringKey = TableLock::Write(1024).into();
assert_eq!(
string_key,
StringKey::Exclusive(format!("{}/{}", TABLE_LOCK_PREFIX, 1024))
);
// The table name lock
let string_key: StringKey = TableNameLock::new("foo", "bar", "baz").into();
assert_eq!(
string_key,
StringKey::Exclusive(format!("{}/{}", TABLE_NAME_LOCK_PREFIX, "foo.bar.baz"))
);
// The region lock
let region_id = RegionId::new(1024, 1);
let string_key: StringKey = RegionLock::Read(region_id).into();
assert_eq!(
string_key,
StringKey::Share(format!("{}/{}", REGION_LOCK_PREFIX, region_id.as_u64()))
);
let string_key: StringKey = RegionLock::Write(region_id).into();
assert_eq!(
string_key,
StringKey::Exclusive(format!("{}/{}", REGION_LOCK_PREFIX, region_id.as_u64()))
);
}
}

View File

@@ -16,36 +16,43 @@ use lazy_static::lazy_static;
use prometheus::*;
lazy_static! {
pub static ref METRIC_META_TXN_REQUEST: HistogramVec =
register_histogram_vec!("meta_txn_request", "meta txn request", &["target", "op"]).unwrap();
pub static ref METRIC_META_TXN_REQUEST: HistogramVec = register_histogram_vec!(
"greptime_meta_txn_request",
"meta txn request",
&["target", "op"]
)
.unwrap();
pub static ref METRIC_META_CREATE_CATALOG: Histogram =
register_histogram!("meta_create_catalog", "meta create catalog").unwrap();
pub static ref METRIC_META_CREATE_CATALOG_COUNTER: IntCounter =
register_int_counter!("meta_create_catalog_counter", "meta create catalog").unwrap();
register_histogram!("greptime_meta_create_catalog", "meta create catalog").unwrap();
pub static ref METRIC_META_CREATE_CATALOG_COUNTER: IntCounter = register_int_counter!(
"greptime_meta_create_catalog_counter",
"meta create catalog"
)
.unwrap();
pub static ref METRIC_META_CREATE_SCHEMA: Histogram =
register_histogram!("meta_create_schema", "meta create schema").unwrap();
register_histogram!("greptime_meta_create_schema", "meta create schema").unwrap();
pub static ref METRIC_META_CREATE_SCHEMA_COUNTER: IntCounter =
register_int_counter!("meta_create_schema_counter", "meta create schema").unwrap();
register_int_counter!("greptime_meta_create_schema_counter", "meta create schema").unwrap();
pub static ref METRIC_META_PROCEDURE_CREATE_TABLE: HistogramVec = register_histogram_vec!(
"meta_procedure_create_table",
"greptime_meta_procedure_create_table",
"meta procedure create table",
&["step"]
)
.unwrap();
pub static ref METRIC_META_PROCEDURE_DROP_TABLE: HistogramVec = register_histogram_vec!(
"meta_procedure_drop_table",
"greptime_meta_procedure_drop_table",
"meta procedure drop table",
&["step"]
)
.unwrap();
pub static ref METRIC_META_PROCEDURE_ALTER_TABLE: HistogramVec = register_histogram_vec!(
"meta_procedure_alter_table",
"greptime_meta_procedure_alter_table",
"meta procedure alter table",
&["step"]
)
.unwrap();
pub static ref METRIC_META_PROCEDURE_TRUNCATE_TABLE: HistogramVec = register_histogram_vec!(
"meta_procedure_truncate_table",
"greptime_meta_procedure_truncate_table",
"meta procedure truncate table",
&["step"]
)

View File

@@ -30,7 +30,7 @@ use crate::peer::Peer;
use crate::table_name::TableName;
use crate::DatanodeId;
pub fn region_distribution(region_routes: &[RegionRoute]) -> Result<RegionDistribution> {
pub fn region_distribution(region_routes: &[RegionRoute]) -> RegionDistribution {
let mut regions_id_map = RegionDistribution::new();
for route in region_routes.iter() {
if let Some(peer) = route.leader_peer.as_ref() {
@@ -42,7 +42,7 @@ pub fn region_distribution(region_routes: &[RegionRoute]) -> Result<RegionDistri
// id asc
regions.sort()
}
Ok(regions_id_map)
regions_id_map
}
#[derive(Debug, Clone, Deserialize, Serialize, PartialEq)]
@@ -123,11 +123,12 @@ pub fn convert_to_region_leader_status_map(
pub fn find_region_leader(
region_routes: &[RegionRoute],
region_number: RegionNumber,
) -> Option<&Peer> {
) -> Option<Peer> {
region_routes
.iter()
.find(|x| x.region.id.region_number() == region_number)
.and_then(|r| r.leader_peer.as_ref())
.cloned()
}
pub fn find_leader_regions(region_routes: &[RegionRoute], datanode: &Peer) -> Vec<RegionNumber> {

View File

@@ -18,12 +18,11 @@ pub mod options_allocator;
use std::collections::HashMap;
use common_config::wal::StandaloneWalConfig;
use common_config::WAL_OPTIONS_KEY;
use serde::{Deserialize, Serialize};
use serde_with::with_prefix;
use store_api::storage::{RegionId, RegionNumber};
use crate::error::Result;
use crate::wal::kafka::KafkaConfig;
pub use crate::wal::kafka::Topic as KafkaWalTopic;
pub use crate::wal::options_allocator::{
allocate_region_wal_options, WalOptionsAllocator, WalOptionsAllocatorRef,
};
@@ -40,7 +39,7 @@ pub enum WalConfig {
impl From<StandaloneWalConfig> for WalConfig {
fn from(value: StandaloneWalConfig) -> Self {
match value {
StandaloneWalConfig::RaftEngine(config) => WalConfig::RaftEngine,
StandaloneWalConfig::RaftEngine(_) => WalConfig::RaftEngine,
StandaloneWalConfig::Kafka(config) => WalConfig::Kafka(KafkaConfig {
broker_endpoints: config.base.broker_endpoints,
num_topics: config.num_topics,
@@ -55,6 +54,16 @@ impl From<StandaloneWalConfig> for WalConfig {
}
}
pub fn prepare_wal_option(
options: &mut HashMap<String, String>,
region_id: RegionId,
region_wal_options: &HashMap<RegionNumber, String>,
) {
if let Some(wal_options) = region_wal_options.get(&region_id.region_number()) {
options.insert(WAL_OPTIONS_KEY.to_string(), wal_options.clone());
}
}
#[cfg(test)]
mod tests {
use std::time::Duration;
@@ -88,7 +97,6 @@ mod tests {
num_topics = 32
selector_type = "round_robin"
topic_name_prefix = "greptimedb_wal_topic"
num_partitions = 1
replication_factor = 1
create_topic_timeout = "30s"
backoff_init = "500ms"

View File

@@ -12,21 +12,21 @@
// See the License for the specific language governing permissions and
// limitations under the License.
pub mod topic;
#[cfg(any(test, feature = "testing"))]
pub mod test_util;
pub mod topic_manager;
pub mod topic_selector;
use std::time::Duration;
use common_config::wal::kafka::{kafka_backoff, KafkaBackoffConfig, TopicSelectorType};
use common_config::wal::StandaloneWalConfig;
use serde::{Deserialize, Serialize};
pub use crate::wal::kafka::topic::Topic;
pub use crate::wal::kafka::topic_manager::TopicManager;
/// Configurations for kafka wal.
#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)]
#[serde(default)]
pub struct KafkaConfig {
/// The broker endpoints of the Kafka cluster.
pub broker_endpoints: Vec<String>,
@@ -40,7 +40,7 @@ pub struct KafkaConfig {
pub num_partitions: i32,
/// The replication factor of each topic.
pub replication_factor: i16,
/// Above which a topic creation operation will be cancelled.
/// The timeout of topic creation.
#[serde(with = "humantime_serde")]
pub create_topic_timeout: Duration,
/// The backoff config.

View File

@@ -0,0 +1,33 @@
// 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_telemetry::warn;
use futures_util::future::BoxFuture;
pub async fn run_test_with_kafka_wal<F>(test: F)
where
F: FnOnce(Vec<String>) -> BoxFuture<'static, ()>,
{
let Ok(endpoints) = std::env::var("GT_KAFKA_ENDPOINTS") else {
warn!("The endpoints is empty, skipping the test");
return;
};
let endpoints = endpoints
.split(',')
.map(|s| s.trim().to_string())
.collect::<Vec<_>>();
test(endpoints).await
}

View File

@@ -14,35 +14,39 @@
use std::collections::HashSet;
use std::sync::Arc;
use std::time::Duration;
use common_config::wal::kafka::TopicSelectorType;
use common_telemetry::{debug, error, info};
use common_telemetry::{error, info};
use rskafka::client::controller::ControllerClient;
use rskafka::client::error::Error as RsKafkaError;
use rskafka::client::error::ProtocolError::TopicAlreadyExists;
use rskafka::client::ClientBuilder;
use rskafka::client::partition::{Compression, UnknownTopicHandling};
use rskafka::client::{Client, ClientBuilder};
use rskafka::record::Record;
use rskafka::BackoffConfig;
use snafu::{ensure, AsErrorSource, ResultExt};
use snafu::{ensure, ResultExt};
use crate::error::{
BuildKafkaClientSnafu, BuildKafkaCtrlClientSnafu, CreateKafkaWalTopicSnafu, DecodeJsonSnafu,
EncodeJsonSnafu, InvalidNumTopicsSnafu, Result,
BuildKafkaClientSnafu, BuildKafkaCtrlClientSnafu, BuildKafkaPartitionClientSnafu,
CreateKafkaWalTopicSnafu, DecodeJsonSnafu, EncodeJsonSnafu, InvalidNumTopicsSnafu,
ProduceRecordSnafu, Result,
};
use crate::kv_backend::KvBackendRef;
use crate::rpc::store::PutRequest;
use crate::wal::kafka::topic::Topic;
use crate::wal::kafka::topic_selector::{RoundRobinTopicSelector, TopicSelectorRef};
use crate::wal::kafka::KafkaConfig;
const CREATED_TOPICS_KEY: &str = "__created_wal_topics/kafka/";
// Each topic only has one partition for now.
// The `DEFAULT_PARTITION` refers to the index of the partition.
const DEFAULT_PARTITION: i32 = 0;
/// Manages topic initialization and selection.
pub struct TopicManager {
config: KafkaConfig,
// TODO(niebayes): maybe add a guard to ensure all topics in the topic pool are created.
topic_pool: Vec<Topic>,
topic_selector: TopicSelectorRef,
pub(crate) topic_pool: Vec<String>,
pub(crate) topic_selector: TopicSelectorRef,
kv_backend: KvBackendRef,
}
@@ -81,7 +85,7 @@ impl TopicManager {
let created_topics = Self::restore_created_topics(&self.kv_backend)
.await?
.into_iter()
.collect::<HashSet<Topic>>();
.collect::<HashSet<String>>();
// Creates missing topics.
let to_be_created = topics
@@ -103,7 +107,7 @@ impl TopicManager {
}
/// Tries to create topics specified by indexes in `to_be_created`.
async fn try_create_topics(&self, topics: &[Topic], to_be_created: &[usize]) -> Result<()> {
async fn try_create_topics(&self, topics: &[String], to_be_created: &[usize]) -> Result<()> {
// Builds an kafka controller client for creating topics.
let backoff_config = BackoffConfig {
init_backoff: self.config.backoff.init,
@@ -117,31 +121,62 @@ impl TopicManager {
.await
.with_context(|_| BuildKafkaClientSnafu {
broker_endpoints: self.config.broker_endpoints.clone(),
})?
})?;
let control_client = client
.controller_client()
.context(BuildKafkaCtrlClientSnafu)?;
// Try to create missing topics.
let tasks = to_be_created
.iter()
.map(|i| self.try_create_topic(&topics[*i], &client))
.map(|i| async {
self.try_create_topic(&topics[*i], &control_client).await?;
self.try_append_noop_record(&topics[*i], &client).await?;
Ok(())
})
.collect::<Vec<_>>();
futures::future::try_join_all(tasks).await.map(|_| ())
}
/// Selects one topic from the topic pool through the topic selector.
pub fn select(&self) -> Result<&Topic> {
pub fn select(&self) -> Result<&String> {
self.topic_selector.select(&self.topic_pool)
}
/// Selects a batch of topics from the topic pool through the topic selector.
pub fn select_batch(&self, num_topics: usize) -> Result<Vec<&Topic>> {
pub fn select_batch(&self, num_topics: usize) -> Result<Vec<&String>> {
(0..num_topics)
.map(|_| self.topic_selector.select(&self.topic_pool))
.collect()
}
async fn try_create_topic(&self, topic: &Topic, client: &ControllerClient) -> Result<()> {
async fn try_append_noop_record(&self, topic: &String, client: &Client) -> Result<()> {
let partition_client = client
.partition_client(topic, DEFAULT_PARTITION, UnknownTopicHandling::Retry)
.await
.context(BuildKafkaPartitionClientSnafu {
topic,
partition: DEFAULT_PARTITION,
})?;
partition_client
.produce(
vec![Record {
key: None,
value: None,
timestamp: chrono::Utc::now(),
headers: Default::default(),
}],
Compression::NoCompression,
)
.await
.context(ProduceRecordSnafu { topic })?;
Ok(())
}
async fn try_create_topic(&self, topic: &String, client: &ControllerClient) -> Result<()> {
match client
.create_topic(
topic.clone(),
@@ -167,7 +202,7 @@ impl TopicManager {
}
}
async fn restore_created_topics(kv_backend: &KvBackendRef) -> Result<Vec<Topic>> {
async fn restore_created_topics(kv_backend: &KvBackendRef) -> Result<Vec<String>> {
kv_backend
.get(CREATED_TOPICS_KEY.as_bytes())
.await?
@@ -177,7 +212,7 @@ impl TopicManager {
)
}
async fn persist_created_topics(topics: &[Topic], kv_backend: &KvBackendRef) -> Result<()> {
async fn persist_created_topics(topics: &[String], kv_backend: &KvBackendRef) -> Result<()> {
let raw_topics = serde_json::to_vec(topics).context(EncodeJsonSnafu)?;
kv_backend
.put(PutRequest {
@@ -202,13 +237,9 @@ impl TopicManager {
#[cfg(test)]
mod tests {
use std::env;
use common_telemetry::info;
use super::*;
use crate::kv_backend::memory::MemoryKvBackend;
use crate::kv_backend::{self};
use crate::wal::kafka::test_util::run_test_with_kafka_wal;
// Tests that topics can be successfully persisted into the kv backend and can be successfully restored from the kv backend.
#[tokio::test]
@@ -235,26 +266,60 @@ mod tests {
assert_eq!(topics, restored_topics);
}
/// Tests that the topic manager could allocate topics correctly.
#[tokio::test]
async fn test_topic_manager() {
let endpoints = env::var("GT_KAFKA_ENDPOINTS").unwrap_or_default();
common_telemetry::init_default_ut_logging();
async fn test_alloc_topics() {
run_test_with_kafka_wal(|broker_endpoints| {
Box::pin(async {
// Constructs topics that should be created.
let topics = (0..256)
.map(|i| format!("test_alloc_topics_{}_{}", i, uuid::Uuid::new_v4()))
.collect::<Vec<_>>();
if endpoints.is_empty() {
info!("The endpoints is empty, skipping the test.");
return;
}
// TODO: supports topic prefix
let kv_backend = Arc::new(MemoryKvBackend::new());
let config = KafkaConfig {
replication_factor: 1,
broker_endpoints: endpoints
.split(',')
.map(|s| s.to_string())
.collect::<Vec<_>>(),
..Default::default()
};
let manager = TopicManager::new(config, kv_backend);
manager.start().await.unwrap();
// Creates a topic manager.
let config = KafkaConfig {
replication_factor: broker_endpoints.len() as i16,
broker_endpoints,
..Default::default()
};
let kv_backend = Arc::new(MemoryKvBackend::new()) as KvBackendRef;
let mut manager = TopicManager::new(config.clone(), kv_backend);
// Replaces the default topic pool with the constructed topics.
manager.topic_pool = topics.clone();
// Replaces the default selector with a round-robin selector without shuffled.
manager.topic_selector = Arc::new(RoundRobinTopicSelector::default());
manager.start().await.unwrap();
// Selects exactly the number of `num_topics` topics one by one.
let got = (0..topics.len())
.map(|_| manager.select().unwrap())
.cloned()
.collect::<Vec<_>>();
assert_eq!(got, topics);
// Selects exactly the number of `num_topics` topics in a batching manner.
let got = manager
.select_batch(topics.len())
.unwrap()
.into_iter()
.map(ToString::to_string)
.collect::<Vec<_>>();
assert_eq!(got, topics);
// Selects more than the number of `num_topics` topics.
let got = manager
.select_batch(2 * topics.len())
.unwrap()
.into_iter()
.map(ToString::to_string)
.collect::<Vec<_>>();
let expected = vec![topics.clone(); 2]
.into_iter()
.flatten()
.collect::<Vec<_>>();
assert_eq!(got, expected);
})
})
.await;
}
}

View File

@@ -16,16 +16,14 @@ use std::sync::atomic::{AtomicUsize, Ordering};
use std::sync::Arc;
use rand::Rng;
use serde::{Deserialize, Serialize};
use snafu::ensure;
use crate::error::{EmptyTopicPoolSnafu, Result};
use crate::wal::kafka::topic::Topic;
/// Controls topic selection.
pub(crate) trait TopicSelector: Send + Sync {
/// Selects a topic from the topic pool.
fn select<'a>(&self, topic_pool: &'a [Topic]) -> Result<&'a Topic>;
fn select<'a>(&self, topic_pool: &'a [String]) -> Result<&'a String>;
}
/// Arc wrapper of TopicSelector.
@@ -49,7 +47,7 @@ impl RoundRobinTopicSelector {
}
impl TopicSelector for RoundRobinTopicSelector {
fn select<'a>(&self, topic_pool: &'a [Topic]) -> Result<&'a Topic> {
fn select<'a>(&self, topic_pool: &'a [String]) -> Result<&'a String> {
ensure!(!topic_pool.is_empty(), EmptyTopicPoolSnafu);
let which = self.cursor.fetch_add(1, Ordering::Relaxed) % topic_pool.len();
Ok(&topic_pool[which])
@@ -60,6 +58,14 @@ impl TopicSelector for RoundRobinTopicSelector {
mod tests {
use super::*;
/// Tests that a selector behaves as expected when the given topic pool is empty.
#[test]
fn test_empty_topic_pool() {
let topic_pool = vec![];
let selector = RoundRobinTopicSelector::default();
assert!(selector.select(&topic_pool).is_err());
}
#[test]
fn test_round_robin_topic_selector() {
let topic_pool: Vec<_> = [0, 1, 2].into_iter().map(|v| v.to_string()).collect();

View File

@@ -107,14 +107,16 @@ pub fn allocate_region_wal_options(
mod tests {
use super::*;
use crate::kv_backend::memory::MemoryKvBackend;
use crate::wal::kafka::test_util::run_test_with_kafka_wal;
use crate::wal::kafka::topic_selector::RoundRobinTopicSelector;
use crate::wal::kafka::KafkaConfig;
// Tests the wal options allocator could successfully allocate raft-engine wal options.
// Note: tests for allocator with kafka are integration tests.
#[tokio::test]
async fn test_allocator_with_raft_engine() {
let kv_backend = Arc::new(MemoryKvBackend::new()) as KvBackendRef;
let wal_config = WalConfig::RaftEngine;
let mut allocator = WalOptionsAllocator::new(wal_config, kv_backend);
let allocator = WalOptionsAllocator::new(wal_config, kv_backend);
allocator.start().await.unwrap();
let num_regions = 32;
@@ -128,4 +130,49 @@ mod tests {
.collect();
assert_eq!(got, expected);
}
// Tests that the wal options allocator could successfully allocate Kafka wal options.
#[tokio::test]
async fn test_allocator_with_kafka() {
run_test_with_kafka_wal(|broker_endpoints| {
Box::pin(async {
let topics = (0..256)
.map(|i| format!("test_allocator_with_kafka_{}_{}", i, uuid::Uuid::new_v4()))
.collect::<Vec<_>>();
// Creates a topic manager.
let config = KafkaConfig {
replication_factor: broker_endpoints.len() as i16,
broker_endpoints,
..Default::default()
};
let kv_backend = Arc::new(MemoryKvBackend::new()) as KvBackendRef;
let mut topic_manager = KafkaTopicManager::new(config.clone(), kv_backend);
// Replaces the default topic pool with the constructed topics.
topic_manager.topic_pool = topics.clone();
// Replaces the default selector with a round-robin selector without shuffled.
topic_manager.topic_selector = Arc::new(RoundRobinTopicSelector::default());
// Creates an options allocator.
let allocator = WalOptionsAllocator::Kafka(topic_manager);
allocator.start().await.unwrap();
let num_regions = 32;
let regions = (0..num_regions).collect::<Vec<_>>();
let got = allocate_region_wal_options(regions.clone(), &allocator).unwrap();
// Check the allocated wal options contain the expected topics.
let expected = (0..num_regions)
.map(|i| {
let options = WalOptions::Kafka(KafkaWalOptions {
topic: topics[i as usize].clone(),
});
(i, serde_json::to_string(&options).unwrap())
})
.collect::<HashMap<_, _>>();
assert_eq!(got, expected);
})
})
.await;
}
}

View File

@@ -26,6 +26,6 @@ pub mod watcher;
pub use crate::error::{Error, Result};
pub use crate::procedure::{
BoxedProcedure, Context, ContextProvider, LockKey, Procedure, ProcedureId, ProcedureManager,
ProcedureManagerRef, ProcedureState, ProcedureWithId, Status,
ProcedureManagerRef, ProcedureState, ProcedureWithId, Status, StringKey,
};
pub use crate::watcher::Watcher;

View File

@@ -12,8 +12,8 @@
// See the License for the specific language governing permissions and
// limitations under the License.
mod lock;
mod runner;
mod rwlock;
use std::collections::{HashMap, VecDeque};
use std::sync::atomic::{AtomicBool, Ordering};
@@ -29,11 +29,11 @@ use snafu::{ensure, ResultExt};
use tokio::sync::watch::{self, Receiver, Sender};
use tokio::sync::{Mutex as TokioMutex, Notify};
use self::rwlock::KeyRwLock;
use crate::error::{
DuplicateProcedureSnafu, Error, LoaderConflictSnafu, ManagerNotStartSnafu, Result,
StartRemoveOutdatedMetaTaskSnafu, StopRemoveOutdatedMetaTaskSnafu,
};
use crate::local::lock::LockMap;
use crate::local::runner::Runner;
use crate::procedure::BoxedProcedureLoader;
use crate::store::{ProcedureMessage, ProcedureStore, StateStoreRef};
@@ -57,8 +57,6 @@ const META_TTL: Duration = Duration::from_secs(60 * 10);
pub(crate) struct ProcedureMeta {
/// Id of this procedure.
id: ProcedureId,
/// Notify to wait for a lock.
lock_notify: Notify,
/// Parent procedure id.
parent_id: Option<ProcedureId>,
/// Notify to wait for subprocedures.
@@ -78,7 +76,6 @@ impl ProcedureMeta {
let (state_sender, state_receiver) = watch::channel(ProcedureState::Running);
ProcedureMeta {
id,
lock_notify: Notify::new(),
parent_id,
child_notify: Notify::new(),
lock_key,
@@ -131,7 +128,7 @@ struct LoadedProcedure {
pub(crate) struct ManagerContext {
/// Procedure loaders. The key is the type name of the procedure which the loader returns.
loaders: Mutex<HashMap<String, BoxedProcedureLoader>>,
lock_map: LockMap,
key_lock: KeyRwLock<String>,
procedures: RwLock<HashMap<ProcedureId, ProcedureMetaRef>>,
/// Messages loaded from the procedure store.
messages: Mutex<HashMap<ProcedureId, ProcedureMessage>>,
@@ -152,8 +149,8 @@ impl ManagerContext {
/// Returns a new [ManagerContext].
fn new() -> ManagerContext {
ManagerContext {
key_lock: KeyRwLock::new(),
loaders: Mutex::new(HashMap::new()),
lock_map: LockMap::new(),
procedures: RwLock::new(HashMap::new()),
messages: Mutex::new(HashMap::new()),
finished_procedures: Mutex::new(VecDeque::new()),
@@ -850,7 +847,7 @@ mod tests {
assert!(manager.procedure_watcher(procedure_id).is_none());
let mut procedure = ProcedureToLoad::new("submit");
procedure.lock_key = LockKey::single("test.submit");
procedure.lock_key = LockKey::single_exclusive("test.submit");
assert!(manager
.submit(ProcedureWithId {
id: procedure_id,
@@ -918,7 +915,7 @@ mod tests {
}
fn lock_key(&self) -> LockKey {
LockKey::single("test.submit")
LockKey::single_exclusive("test.submit")
}
}
@@ -955,7 +952,7 @@ mod tests {
let manager = LocalManager::new(config, state_store);
let mut procedure = ProcedureToLoad::new("submit");
procedure.lock_key = LockKey::single("test.submit");
procedure.lock_key = LockKey::single_exclusive("test.submit");
let procedure_id = ProcedureId::random();
assert_matches!(
manager
@@ -986,7 +983,7 @@ mod tests {
manager.start().await.unwrap();
let mut procedure = ProcedureToLoad::new("submit");
procedure.lock_key = LockKey::single("test.submit");
procedure.lock_key = LockKey::single_exclusive("test.submit");
let procedure_id = ProcedureId::random();
assert!(manager
.submit(ProcedureWithId {
@@ -1018,7 +1015,7 @@ mod tests {
manager.manager_ctx.set_running();
let mut procedure = ProcedureToLoad::new("submit");
procedure.lock_key = LockKey::single("test.submit");
procedure.lock_key = LockKey::single_exclusive("test.submit");
let procedure_id = ProcedureId::random();
assert!(manager
.submit(ProcedureWithId {
@@ -1041,7 +1038,7 @@ mod tests {
// The remove_outdated_meta method has been stopped, so any procedure meta-data will not be automatically removed.
manager.stop().await.unwrap();
let mut procedure = ProcedureToLoad::new("submit");
procedure.lock_key = LockKey::single("test.submit");
procedure.lock_key = LockKey::single_exclusive("test.submit");
let procedure_id = ProcedureId::random();
manager.manager_ctx.set_running();
@@ -1063,7 +1060,7 @@ mod tests {
// After restart
let mut procedure = ProcedureToLoad::new("submit");
procedure.lock_key = LockKey::single("test.submit");
procedure.lock_key = LockKey::single_exclusive("test.submit");
let procedure_id = ProcedureId::random();
assert!(manager
.submit(ProcedureWithId {

View File

@@ -1,214 +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::{HashMap, VecDeque};
use std::sync::RwLock;
use crate::local::ProcedureMetaRef;
use crate::ProcedureId;
/// A lock entry.
#[derive(Debug)]
struct Lock {
/// Current lock owner.
owner: ProcedureMetaRef,
/// Waiter procedures.
waiters: VecDeque<ProcedureMetaRef>,
}
impl Lock {
/// Returns a [Lock] with specific `owner` procedure.
fn from_owner(owner: ProcedureMetaRef) -> Lock {
Lock {
owner,
waiters: VecDeque::new(),
}
}
/// Try to pop a waiter from the waiter list, set it as owner
/// and wake up the new owner.
///
/// Returns false if there is no waiter in the waiter list.
fn switch_owner(&mut self) -> bool {
if let Some(waiter) = self.waiters.pop_front() {
// Update owner.
self.owner = waiter.clone();
// We need to use notify_one() since the waiter may have not called `notified()` yet.
waiter.lock_notify.notify_one();
true
} else {
false
}
}
}
/// Manages lock entries for procedures.
pub(crate) struct LockMap {
locks: RwLock<HashMap<String, Lock>>,
}
impl LockMap {
/// Returns a new [LockMap].
pub(crate) fn new() -> LockMap {
LockMap {
locks: RwLock::new(HashMap::new()),
}
}
/// Acquire lock by `key` for procedure with specific `meta`.
///
/// Though `meta` is cloneable, callers must ensure that only one `meta`
/// is acquiring and holding the lock at the same time.
///
/// # Panics
/// Panics if the procedure acquires the lock recursively.
pub(crate) async fn acquire_lock(&self, key: &str, meta: ProcedureMetaRef) {
assert!(!self.hold_lock(key, meta.id));
{
let mut locks = self.locks.write().unwrap();
if let Some(lock) = locks.get_mut(key) {
// Lock already exists, but we don't expect that a procedure acquires
// the same lock again.
assert_ne!(lock.owner.id, meta.id);
// Add this procedure to the waiter list. Here we don't check
// whether the procedure is already in the waiter list as we
// expect that a procedure should not wait for two lock simultaneously.
lock.waiters.push_back(meta.clone());
} else {
let _ = locks.insert(key.to_string(), Lock::from_owner(meta));
return;
}
}
// Wait for notify.
meta.lock_notify.notified().await;
assert!(self.hold_lock(key, meta.id));
}
/// Release lock by `key`.
pub(crate) fn release_lock(&self, key: &str, procedure_id: ProcedureId) {
let mut locks = self.locks.write().unwrap();
if let Some(lock) = locks.get_mut(key) {
if lock.owner.id != procedure_id {
// This is not the lock owner.
return;
}
if !lock.switch_owner() {
// No body waits for this lock, we can remove the lock entry.
let _ = locks.remove(key);
}
}
}
/// Returns true if the procedure with specific `procedure_id` holds the
/// lock of `key`.
fn hold_lock(&self, key: &str, procedure_id: ProcedureId) -> bool {
let locks = self.locks.read().unwrap();
locks
.get(key)
.map(|lock| lock.owner.id == procedure_id)
.unwrap_or(false)
}
/// Returns true if the procedure is waiting for the lock `key`.
#[cfg(test)]
fn waiting_lock(&self, key: &str, procedure_id: ProcedureId) -> bool {
let locks = self.locks.read().unwrap();
locks
.get(key)
.map(|lock| lock.waiters.iter().any(|meta| meta.id == procedure_id))
.unwrap_or(false)
}
}
#[cfg(test)]
mod tests {
use std::sync::Arc;
use super::*;
use crate::local::test_util;
#[test]
fn test_lock_no_waiter() {
let meta = Arc::new(test_util::procedure_meta_for_test());
let mut lock = Lock::from_owner(meta);
assert!(!lock.switch_owner());
}
#[tokio::test]
async fn test_lock_with_waiter() {
let owner = Arc::new(test_util::procedure_meta_for_test());
let mut lock = Lock::from_owner(owner);
let waiter = Arc::new(test_util::procedure_meta_for_test());
lock.waiters.push_back(waiter.clone());
assert!(lock.switch_owner());
assert!(lock.waiters.is_empty());
waiter.lock_notify.notified().await;
assert_eq!(lock.owner.id, waiter.id);
}
#[tokio::test]
async fn test_lock_map() {
let key = "hello";
let owner = Arc::new(test_util::procedure_meta_for_test());
let lock_map = Arc::new(LockMap::new());
lock_map.acquire_lock(key, owner.clone()).await;
let waiter = Arc::new(test_util::procedure_meta_for_test());
let waiter_id = waiter.id;
// Waiter release the lock, this should not take effect.
lock_map.release_lock(key, waiter_id);
let lock_map2 = lock_map.clone();
let owner_id = owner.id;
let handle = tokio::spawn(async move {
assert!(lock_map2.hold_lock(key, owner_id));
assert!(!lock_map2.hold_lock(key, waiter_id));
// Waiter wait for lock.
lock_map2.acquire_lock(key, waiter.clone()).await;
assert!(lock_map2.hold_lock(key, waiter_id));
});
// Owner still holds the lock.
assert!(lock_map.hold_lock(key, owner_id));
// Wait until the waiter acquired the lock
while !lock_map.waiting_lock(key, waiter_id) {
tokio::time::sleep(std::time::Duration::from_millis(5)).await;
}
// Release lock
lock_map.release_lock(key, owner_id);
assert!(!lock_map.hold_lock(key, owner_id));
// Wait for task.
handle.await.unwrap();
// The waiter should hold the lock now.
assert!(lock_map.hold_lock(key, waiter_id));
lock_map.release_lock(key, waiter_id);
}
}

View File

@@ -19,8 +19,10 @@ use backon::{BackoffBuilder, ExponentialBuilder};
use common_telemetry::logging;
use tokio::time;
use super::rwlock::OwnedKeyRwLockGuard;
use crate::error::{self, ProcedurePanicSnafu, Result};
use crate::local::{ManagerContext, ProcedureMeta, ProcedureMetaRef};
use crate::procedure::StringKey;
use crate::store::ProcedureStore;
use crate::ProcedureState::Retrying;
use crate::{BoxedProcedure, Context, Error, ProcedureId, ProcedureState, ProcedureWithId, Status};
@@ -56,6 +58,7 @@ impl ExecResult {
struct ProcedureGuard {
meta: ProcedureMetaRef,
manager_ctx: Arc<ManagerContext>,
key_guards: Vec<OwnedKeyRwLockGuard>,
finish: bool,
}
@@ -65,6 +68,7 @@ impl ProcedureGuard {
ProcedureGuard {
meta,
manager_ctx,
key_guards: vec![],
finish: false,
}
}
@@ -95,10 +99,15 @@ impl Drop for ProcedureGuard {
self.manager_ctx.notify_by_subprocedure(parent_id);
}
// Release lock in reverse order.
for key in self.meta.lock_key.keys_to_unlock() {
self.manager_ctx.lock_map.release_lock(key, self.meta.id);
// Drops the key guards in the reverse order.
while !self.key_guards.is_empty() {
self.key_guards.pop();
}
// Clean the staled locks.
self.manager_ctx
.key_lock
.clean_keys(self.meta.lock_key.keys_to_lock().map(|k| k.as_string()));
}
}
@@ -121,7 +130,7 @@ impl Runner {
/// Run the procedure.
pub(crate) async fn run(mut self) {
// Ensure we can update the procedure state.
let guard = ProcedureGuard::new(self.meta.clone(), self.manager_ctx.clone());
let mut guard = ProcedureGuard::new(self.meta.clone(), self.manager_ctx.clone());
logging::info!(
"Runner {}-{} starts",
@@ -133,10 +142,14 @@ impl Runner {
// recursive locking by adding a root procedure id to the meta.
for key in self.meta.lock_key.keys_to_lock() {
// Acquire lock for each key.
self.manager_ctx
.lock_map
.acquire_lock(key, self.meta.clone())
.await;
let key_guard = match key {
StringKey::Share(key) => self.manager_ctx.key_lock.read(key.clone()).await.into(),
StringKey::Exclusive(key) => {
self.manager_ctx.key_lock.write(key.clone()).await.into()
}
};
guard.key_guards.push(key_guard);
}
// Execute the procedure. We need to release the lock whenever the the execution
@@ -604,7 +617,7 @@ mod tests {
};
let normal = ProcedureAdapter {
data: "normal".to_string(),
lock_key: LockKey::single("catalog.schema.table"),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
exec_fn,
};
@@ -665,7 +678,7 @@ mod tests {
};
let suspend = ProcedureAdapter {
data: "suspend".to_string(),
lock_key: LockKey::single("catalog.schema.table"),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
exec_fn,
};
@@ -697,7 +710,7 @@ mod tests {
};
let child = ProcedureAdapter {
data: "child".to_string(),
lock_key: LockKey::new(keys.iter().map(|k| k.to_string())),
lock_key: LockKey::new_exclusive(keys.iter().map(|k| k.to_string())),
exec_fn,
};
@@ -765,7 +778,7 @@ mod tests {
};
let parent = ProcedureAdapter {
data: "parent".to_string(),
lock_key: LockKey::single("catalog.schema.table"),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
exec_fn,
};
@@ -784,6 +797,7 @@ mod tests {
runner.manager_ctx = manager_ctx.clone();
runner.run().await;
assert!(manager_ctx.key_lock.is_empty());
// Check child procedures.
for child_id in children_ids {
@@ -810,7 +824,7 @@ mod tests {
let exec_fn = move |_| async move { Ok(Status::Executing { persist: true }) }.boxed();
let normal = ProcedureAdapter {
data: "normal".to_string(),
lock_key: LockKey::single("catalog.schema.table"),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
exec_fn,
};
@@ -851,7 +865,7 @@ mod tests {
|_| async { Err(Error::external(MockError::new(StatusCode::Unexpected))) }.boxed();
let normal = ProcedureAdapter {
data: "fail".to_string(),
lock_key: LockKey::single("catalog.schema.table"),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
exec_fn,
};
@@ -875,7 +889,7 @@ mod tests {
|_| async { Err(Error::external(MockError::new(StatusCode::Unexpected))) }.boxed();
let fail = ProcedureAdapter {
data: "fail".to_string(),
lock_key: LockKey::single("catalog.schema.table"),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
exec_fn,
};
@@ -917,7 +931,7 @@ mod tests {
let retry_later = ProcedureAdapter {
data: "retry_later".to_string(),
lock_key: LockKey::single("catalog.schema.table"),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
exec_fn,
};
@@ -952,7 +966,7 @@ mod tests {
let exceed_max_retry_later = ProcedureAdapter {
data: "exceed_max_retry_later".to_string(),
lock_key: LockKey::single("catalog.schema.table"),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
exec_fn,
};
@@ -993,7 +1007,7 @@ mod tests {
};
let fail = ProcedureAdapter {
data: "fail".to_string(),
lock_key: LockKey::single("catalog.schema.table.region-0"),
lock_key: LockKey::single_exclusive("catalog.schema.table.region-0"),
exec_fn,
};
@@ -1027,7 +1041,7 @@ mod tests {
};
let parent = ProcedureAdapter {
data: "parent".to_string(),
lock_key: LockKey::single("catalog.schema.table"),
lock_key: LockKey::single_exclusive("catalog.schema.table"),
exec_fn,
};
@@ -1042,10 +1056,11 @@ mod tests {
// Manually add this procedure to the manager ctx.
assert!(manager_ctx.try_insert_procedure(meta.clone()));
// Replace the manager ctx.
runner.manager_ctx = manager_ctx;
runner.manager_ctx = manager_ctx.clone();
// Run the runner and execute the procedure.
runner.run().await;
assert!(manager_ctx.key_lock.is_empty());
let err = meta.state().error().unwrap().output_msg();
assert!(err.contains("subprocedure failed"), "{err}");
}

View File

@@ -0,0 +1,247 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use std::collections::HashMap;
use std::hash::Hash;
use std::sync::{Arc, Mutex};
use tokio::sync::{OwnedRwLockReadGuard, OwnedRwLockWriteGuard, RwLock};
pub enum OwnedKeyRwLockGuard {
Read(OwnedRwLockReadGuard<()>),
Write(OwnedRwLockWriteGuard<()>),
}
impl From<OwnedRwLockReadGuard<()>> for OwnedKeyRwLockGuard {
fn from(guard: OwnedRwLockReadGuard<()>) -> Self {
OwnedKeyRwLockGuard::Read(guard)
}
}
impl From<OwnedRwLockWriteGuard<()>> for OwnedKeyRwLockGuard {
fn from(guard: OwnedRwLockWriteGuard<()>) -> Self {
OwnedKeyRwLockGuard::Write(guard)
}
}
/// Locks based on a key, allowing other keys to lock independently.
#[derive(Debug)]
pub struct KeyRwLock<K> {
/// The inner map of locks for specific keys.
inner: Mutex<HashMap<K, Arc<RwLock<()>>>>,
}
impl<K> KeyRwLock<K>
where
K: Eq + Hash + Clone,
{
pub fn new() -> Self {
KeyRwLock {
inner: Default::default(),
}
}
/// Locks the key with shared read access, returning a guard.
pub async fn read(&self, key: K) -> OwnedRwLockReadGuard<()> {
let lock = {
let mut locks = self.inner.lock().unwrap();
locks.entry(key).or_default().clone()
};
lock.read_owned().await
}
/// Locks the key with exclusive write access, returning a guard.
pub async fn write(&self, key: K) -> OwnedRwLockWriteGuard<()> {
let lock = {
let mut locks = self.inner.lock().unwrap();
locks.entry(key).or_default().clone()
};
lock.write_owned().await
}
/// Clean up stale locks.
///
/// Note: It only cleans a lock if
/// - Its strong ref count equals one.
/// - Able to acquire the write lock.
pub fn clean_keys<'a>(&'a self, iter: impl IntoIterator<Item = &'a K>) {
let mut locks = self.inner.lock().unwrap();
let mut keys = Vec::new();
for key in iter {
if let Some(lock) = locks.get(key) {
if lock.try_write().is_ok() {
debug_assert_eq!(Arc::weak_count(lock), 0);
// Ensures nobody keeps this ref.
if Arc::strong_count(lock) == 1 {
keys.push(key);
}
}
}
}
for key in keys {
locks.remove(key);
}
}
}
#[cfg(test)]
impl<K> KeyRwLock<K>
where
K: Eq + Hash + Clone,
{
/// Tries to lock the key with shared read access, returning immediately.
pub fn try_read(&self, key: K) -> Result<OwnedRwLockReadGuard<()>, tokio::sync::TryLockError> {
let lock = {
let mut locks = self.inner.lock().unwrap();
locks.entry(key).or_default().clone()
};
lock.try_read_owned()
}
/// Tries lock this key with exclusive write access, returning immediately.
pub fn try_write(
&self,
key: K,
) -> Result<OwnedRwLockWriteGuard<()>, tokio::sync::TryLockError> {
let lock = {
let mut locks = self.inner.lock().unwrap();
locks.entry(key).or_default().clone()
};
lock.try_write_owned()
}
/// Returns number of keys.
pub fn len(&self) -> usize {
self.inner.lock().unwrap().len()
}
/// Returns true the inner map is empty.
pub fn is_empty(&self) -> bool {
self.len() == 0
}
}
#[cfg(test)]
mod tests {
use super::*;
#[tokio::test]
async fn test_naive() {
let lock_key = KeyRwLock::new();
{
let _guard = lock_key.read("test1").await;
assert_eq!(lock_key.len(), 1);
assert!(lock_key.try_read("test1").is_ok());
assert!(lock_key.try_write("test1").is_err());
}
{
let _guard0 = lock_key.write("test2").await;
let _guard = lock_key.write("test1").await;
assert_eq!(lock_key.len(), 2);
assert!(lock_key.try_read("test1").is_err());
assert!(lock_key.try_write("test1").is_err());
}
assert_eq!(lock_key.len(), 2);
lock_key.clean_keys(&vec!["test1", "test2"]);
assert!(lock_key.is_empty());
let mut guards = Vec::new();
for key in ["test1", "test2"] {
guards.push(lock_key.read(key).await);
}
while !guards.is_empty() {
guards.pop();
}
lock_key.clean_keys(vec![&"test1", &"test2"]);
assert_eq!(lock_key.len(), 0);
}
#[tokio::test]
async fn test_clean_keys() {
let lock_key = KeyRwLock::<&str>::new();
{
let rwlock = {
lock_key
.inner
.lock()
.unwrap()
.entry("test")
.or_default()
.clone()
};
assert_eq!(Arc::strong_count(&rwlock), 2);
let _guard = rwlock.read_owned().await;
{
let inner = lock_key.inner.lock().unwrap();
let rwlock = inner.get("test").unwrap();
assert_eq!(Arc::strong_count(rwlock), 2);
}
}
{
let rwlock = {
lock_key
.inner
.lock()
.unwrap()
.entry("test")
.or_default()
.clone()
};
assert_eq!(Arc::strong_count(&rwlock), 2);
let _guard = rwlock.write_owned().await;
{
let inner = lock_key.inner.lock().unwrap();
let rwlock = inner.get("test").unwrap();
assert_eq!(Arc::strong_count(rwlock), 2);
}
}
{
let inner = lock_key.inner.lock().unwrap();
let rwlock = inner.get("test").unwrap();
assert_eq!(Arc::strong_count(rwlock), 1);
}
// Someone has the ref of the rwlock, but it waits to be granted the lock.
let rwlock = {
lock_key
.inner
.lock()
.unwrap()
.entry("test")
.or_default()
.clone()
};
assert_eq!(Arc::strong_count(&rwlock), 2);
// However, One thread trying to remove the "test" key should have no effect.
lock_key.clean_keys(vec![&"test"]);
// Should get the rwlock.
{
let inner = lock_key.inner.lock().unwrap();
inner.get("test").unwrap();
}
}
}

View File

@@ -116,22 +116,49 @@ impl<T: Procedure + ?Sized> Procedure for Box<T> {
}
}
#[derive(Clone, Debug, PartialEq, Eq, Hash, PartialOrd, Ord)]
pub enum StringKey {
Share(String),
Exclusive(String),
}
/// Keys to identify required locks.
///
/// [LockKey] always sorts keys lexicographically so that they can be acquired
/// in the same order.
// Most procedures should only acquire 1 ~ 2 locks so we use smallvec to hold keys.
/// Most procedures should only acquire 1 ~ 2 locks so we use smallvec to hold keys.
#[derive(Clone, Debug, Default, PartialEq, Eq)]
pub struct LockKey(SmallVec<[String; 2]>);
pub struct LockKey(SmallVec<[StringKey; 2]>);
impl StringKey {
pub fn into_string(self) -> String {
match self {
StringKey::Share(s) => s,
StringKey::Exclusive(s) => s,
}
}
pub fn as_string(&self) -> &String {
match self {
StringKey::Share(s) => s,
StringKey::Exclusive(s) => s,
}
}
}
impl LockKey {
/// Returns a new [LockKey] with only one key.
pub fn single(key: impl Into<String>) -> LockKey {
pub fn single(key: impl Into<StringKey>) -> LockKey {
LockKey(smallvec![key.into()])
}
/// Returns a new [LockKey] with only one key.
pub fn single_exclusive(key: impl Into<String>) -> LockKey {
LockKey(smallvec![StringKey::Exclusive(key.into())])
}
/// Returns a new [LockKey] with keys from specific `iter`.
pub fn new(iter: impl IntoIterator<Item = String>) -> LockKey {
pub fn new(iter: impl IntoIterator<Item = StringKey>) -> LockKey {
let mut vec: SmallVec<_> = iter.into_iter().collect();
vec.sort();
// Dedup keys to avoid acquiring the same key multiple times.
@@ -139,14 +166,14 @@ impl LockKey {
LockKey(vec)
}
/// Returns the keys to lock.
pub fn keys_to_lock(&self) -> impl Iterator<Item = &String> {
self.0.iter()
/// Returns a new [LockKey] with keys from specific `iter`.
pub fn new_exclusive(iter: impl IntoIterator<Item = String>) -> LockKey {
Self::new(iter.into_iter().map(StringKey::Exclusive))
}
/// Returns the keys to unlock.
pub fn keys_to_unlock(&self) -> impl Iterator<Item = &String> {
self.0.iter().rev()
/// Returns the keys to lock.
pub fn keys_to_lock(&self) -> impl Iterator<Item = &StringKey> {
self.0.iter()
}
}
@@ -340,20 +367,25 @@ mod tests {
#[test]
fn test_lock_key() {
let entity = "catalog.schema.my_table";
let key = LockKey::single(entity);
assert_eq!(vec![entity], key.keys_to_lock().collect::<Vec<_>>());
assert_eq!(vec![entity], key.keys_to_unlock().collect::<Vec<_>>());
let key = LockKey::single_exclusive(entity);
assert_eq!(
vec![&StringKey::Exclusive(entity.to_string())],
key.keys_to_lock().collect::<Vec<_>>()
);
let key = LockKey::new([
let key = LockKey::new_exclusive([
"b".to_string(),
"c".to_string(),
"a".to_string(),
"c".to_string(),
]);
assert_eq!(vec!["a", "b", "c"], key.keys_to_lock().collect::<Vec<_>>());
assert_eq!(
vec!["c", "b", "a"],
key.keys_to_unlock().collect::<Vec<_>>()
vec![
&StringKey::Exclusive("a".to_string()),
&StringKey::Exclusive("b".to_string()),
&StringKey::Exclusive("c".to_string())
],
key.keys_to_lock().collect::<Vec<_>>()
);
}

View File

@@ -87,7 +87,7 @@ impl StateStore for ObjectStateStore {
let mut lister = self
.store
.lister_with(path)
.delimiter("")
.recursive(true)
.await
.map_err(|e| {
BoxedError::new(PlainError::new(

View File

@@ -98,7 +98,7 @@ mod tests {
}
fn lock_key(&self) -> LockKey {
LockKey::single("test.submit")
LockKey::single_exclusive("test.submit")
}
}

View File

@@ -18,3 +18,10 @@ pub use crate::columnar_value::ColumnarValue;
pub use crate::function::*;
pub use crate::logical_plan::{create_udf, AggregateFunction, Expr, ScalarUdf};
pub use crate::signature::{Signature, TypeSignature, Volatility};
/// Default timestamp column name for Prometheus metrics.
pub const GREPTIME_TIMESTAMP: &str = "greptime_timestamp";
/// Default value column name for Prometheus metrics.
pub const GREPTIME_VALUE: &str = "greptime_value";
/// Default counter column name for OTLP metrics.
pub const GREPTIME_COUNT: &str = "greptime_count";

View File

@@ -20,13 +20,13 @@ pub const THREAD_NAME_LABEL: &str = "thread_name";
lazy_static! {
pub static ref METRIC_RUNTIME_THREADS_ALIVE: IntGaugeVec = register_int_gauge_vec!(
"runtime_threads_alive",
"greptime_runtime_threads_alive",
"runtime threads alive",
&[THREAD_NAME_LABEL]
)
.unwrap();
pub static ref METRIC_RUNTIME_THREADS_IDLE: IntGaugeVec = register_int_gauge_vec!(
"runtime_threads_idle",
"greptime_runtime_threads_idle",
"runtime threads idle",
&[THREAD_NAME_LABEL]
)

View File

@@ -22,7 +22,7 @@ use prometheus::*;
lazy_static! {
pub static ref PANIC_COUNTER: IntCounter =
register_int_counter!("panic_counter", "panic_counter").unwrap();
register_int_counter!("greptime_panic_counter", "panic_counter").unwrap();
}
pub fn set_panic_hook() {

View File

@@ -5,6 +5,9 @@ edition.workspace = true
license.workspace = true
[dependencies]
client.workspace = true
common-query.workspace = true
common-recordbatch.workspace = true
once_cell.workspace = true
rand.workspace = true
tempfile.workspace = true

View File

@@ -19,6 +19,7 @@ use std::process::Command;
use std::sync::LazyLock;
pub mod ports;
pub mod recordbatch;
pub mod temp_dir;
// Rust is working on an env possibly named `CARGO_WORKSPACE_DIR` to find the root path to the

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 client::Database;
use common_query::Output;
use common_recordbatch::util;
pub enum ExpectedOutput<'a> {
AffectedRows(usize),
QueryResult(&'a str),
}
pub async fn execute_and_check_output(db: &Database, sql: &str, expected: ExpectedOutput<'_>) {
let output = db.sql(sql).await.unwrap();
match (&output, expected) {
(Output::AffectedRows(x), ExpectedOutput::AffectedRows(y)) => {
assert_eq!(*x, y, "actual: \n{}", x)
}
(Output::RecordBatches(_), ExpectedOutput::QueryResult(x))
| (Output::Stream(_), ExpectedOutput::QueryResult(x)) => {
check_output_stream(output, x).await
}
_ => panic!(),
}
}
pub async fn check_output_stream(output: Output, expected: &str) {
let recordbatches = match output {
Output::Stream(stream) => util::collect_batches(stream).await.unwrap(),
Output::RecordBatches(recordbatches) => recordbatches,
_ => unreachable!(),
};
let pretty_print = recordbatches.pretty_print().unwrap();
assert_eq!(pretty_print, expected, "actual: \n{}", pretty_print);
}

View File

@@ -4,6 +4,9 @@ version.workspace = true
edition.workspace = true
license.workspace = true
[features]
testing = []
[dependencies]
api.workspace = true
arrow-flight.workspace = true

View File

@@ -276,7 +276,7 @@ impl Default for DatanodeOptions {
impl DatanodeOptions {
pub fn env_list_keys() -> Option<&'static [&'static str]> {
Some(&["meta_client.metasrv_addrs"])
Some(&["meta_client.metasrv_addrs", "wal.broker_endpoints"])
}
pub fn to_toml_string(&self) -> String {

View File

@@ -22,11 +22,12 @@ use std::sync::Arc;
use catalog::memory::MemoryCatalogManager;
use common_base::Plugins;
use common_config::wal::{KafkaConfig, RaftEngineConfig};
use common_config::{WalConfig, WAL_OPTIONS_KEY};
use common_config::WalConfig;
use common_error::ext::BoxedError;
use common_greptimedb_telemetry::GreptimeDBTelemetryTask;
use common_meta::key::datanode_table::{DatanodeTableManager, DatanodeTableValue};
use common_meta::kv_backend::KvBackendRef;
use common_meta::wal::prepare_wal_option;
pub use common_procedure::options::ProcedureConfig;
use common_runtime::Runtime;
use common_telemetry::{error, info, warn};
@@ -41,10 +42,11 @@ use metric_engine::engine::MetricEngine;
use mito2::config::MitoConfig;
use mito2::engine::MitoEngine;
use object_store::manager::{ObjectStoreManager, ObjectStoreManagerRef};
use object_store::util::normalize_dir;
use object_store::util::{join_dir, normalize_dir};
use query::QueryEngineFactory;
use servers::export_metrics::ExportMetricsTask;
use servers::grpc::{GrpcServer, GrpcServerConfig};
use servers::grpc::builder::GrpcServerBuilder;
use servers::grpc::GrpcServerConfig;
use servers::http::HttpServerBuilder;
use servers::metrics_handler::MetricsHandler;
use servers::server::{start_server, ServerHandler, ServerHandlers};
@@ -59,9 +61,9 @@ use tokio::sync::Notify;
use crate::config::{DatanodeOptions, RegionEngineConfig};
use crate::error::{
CreateDirSnafu, GetMetadataSnafu, MissingKvBackendSnafu, MissingNodeIdSnafu, OpenLogStoreSnafu,
ParseAddrSnafu, Result, RuntimeResourceSnafu, ShutdownInstanceSnafu, ShutdownServerSnafu,
StartServerSnafu,
BuildMitoEngineSnafu, CreateDirSnafu, GetMetadataSnafu, MissingKvBackendSnafu,
MissingNodeIdSnafu, OpenLogStoreSnafu, ParseAddrSnafu, Result, RuntimeResourceSnafu,
ShutdownInstanceSnafu, ShutdownServerSnafu, StartServerSnafu,
};
use crate::event_listener::{
new_region_server_event_channel, NoopRegionServerEventListener, RegionServerEventListenerRef,
@@ -98,7 +100,7 @@ impl Datanode {
self.start_telemetry();
if let Some(t) = self.export_metrics_task.as_ref() {
t.start()
t.start(None).context(StartServerSnafu)?
}
self.start_services().await
@@ -327,15 +329,13 @@ impl DatanodeBuilder {
max_send_message_size: opts.rpc_max_send_message_size.as_bytes() as usize,
};
let server = Box::new(GrpcServer::new(
Some(config),
None,
None,
Some(Arc::new(region_server.clone()) as _),
Some(Arc::new(region_server.clone()) as _),
None,
region_server.runtime(),
));
let server = Box::new(
GrpcServerBuilder::new(region_server.runtime())
.config(config)
.flight_handler(Arc::new(region_server.clone()))
.region_server_handler(Arc::new(region_server.clone()))
.build(),
);
let addr: SocketAddr = opts.rpc_addr.parse().context(ParseAddrSnafu {
addr: &opts.rpc_addr,
@@ -457,20 +457,33 @@ impl DatanodeBuilder {
async fn build_mito_engine(
opts: &DatanodeOptions,
object_store_manager: ObjectStoreManagerRef,
config: MitoConfig,
mut config: MitoConfig,
) -> Result<MitoEngine> {
// Sets write cache path if it is empty.
if config.experimental_write_cache_path.is_empty() {
config.experimental_write_cache_path = join_dir(&opts.storage.data_home, "write_cache");
info!(
"Sets write cache path to {}",
config.experimental_write_cache_path
);
}
let mito_engine = match &opts.wal {
WalConfig::RaftEngine(raft_engine_config) => MitoEngine::new(
config,
Self::build_raft_engine_log_store(&opts.storage.data_home, raft_engine_config)
.await?,
object_store_manager,
),
)
.await
.context(BuildMitoEngineSnafu)?,
WalConfig::Kafka(kafka_config) => MitoEngine::new(
config,
Self::build_kafka_log_store(kafka_config).await?,
object_store_manager,
),
)
.await
.context(BuildMitoEngineSnafu)?,
};
Ok(mito_engine)
}
@@ -538,13 +551,11 @@ async fn open_all_regions(
for region_number in table_value.regions {
// Augments region options with wal options if a wal options is provided.
let mut region_options = table_value.region_info.region_options.clone();
table_value
.region_info
.region_wal_options
.get(&region_number.to_string())
.and_then(|wal_options| {
region_options.insert(WAL_OPTIONS_KEY.to_string(), wal_options.clone())
});
prepare_wal_option(
&mut region_options,
RegionId::new(table_value.table_id, region_number),
&table_value.region_info.region_wal_options,
);
regions.push((
RegionId::new(table_value.table_id, region_number),

View File

@@ -272,6 +272,22 @@ pub enum Error {
location: Location,
source: BoxedError,
},
#[snafu(display(
"Failed to find logical regions in physical region {}",
physical_region_id
))]
FindLogicalRegions {
physical_region_id: RegionId,
source: metric_engine::error::Error,
location: Location,
},
#[snafu(display("Failed to build mito engine"))]
BuildMitoEngine {
source: mito2::error::Error,
location: Location,
},
}
pub type Result<T> = std::result::Result<T, Error>;
@@ -340,6 +356,9 @@ impl ErrorExt for Error {
}
HandleRegionRequest { source, .. } => source.status_code(),
StopRegionEngine { source, .. } => source.status_code(),
FindLogicalRegions { source, .. } => source.status_code(),
BuildMitoEngine { source, .. } => source.status_code(),
}
}

View File

@@ -305,7 +305,7 @@ impl HeartbeatTask {
}
async fn load_region_stats(region_server: &RegionServer) -> Vec<RegionStat> {
let regions = region_server.opened_regions();
let regions = region_server.reportable_regions();
let mut region_stats = Vec::new();
for stat in regions {

View File

@@ -96,6 +96,7 @@ impl HeartbeatResponseHandler for RegionHeartbeatResponseHandler {
Some((_, Instruction::OpenRegion { .. }))
| Some((_, Instruction::CloseRegion { .. }))
| Some((_, Instruction::DowngradeRegion { .. }))
| Some((_, Instruction::UpgradeRegion { .. }))
)
}
@@ -134,7 +135,7 @@ mod tests {
use common_meta::heartbeat::mailbox::{
HeartbeatMailbox, IncomingMessage, MailboxRef, MessageMeta,
};
use common_meta::instruction::{DowngradeRegion, OpenRegion};
use common_meta::instruction::{DowngradeRegion, OpenRegion, UpgradeRegion};
use mito2::config::MitoConfig;
use mito2::engine::MITO_ENGINE_NAME;
use mito2::test_util::{CreateRequestBuilder, TestEnv};
@@ -175,6 +176,44 @@ mod tests {
}
}
#[test]
fn test_is_acceptable() {
common_telemetry::init_default_ut_logging();
let region_server = mock_region_server();
let heartbeat_handler = RegionHeartbeatResponseHandler::new(region_server.clone());
let heartbeat_env = HeartbeatResponseTestEnv::new();
let meta = MessageMeta::new_test(1, "test", "dn-1", "me-0");
// Open region
let region_id = RegionId::new(1024, 1);
let storage_path = "test";
let instruction = open_region_instruction(region_id, storage_path);
assert!(heartbeat_handler
.is_acceptable(&heartbeat_env.create_handler_ctx((meta.clone(), instruction))));
// Close region
let instruction = close_region_instruction(region_id);
assert!(heartbeat_handler
.is_acceptable(&heartbeat_env.create_handler_ctx((meta.clone(), instruction))));
// Downgrade region
let instruction = Instruction::DowngradeRegion(DowngradeRegion {
region_id: RegionId::new(2048, 1),
});
assert!(heartbeat_handler
.is_acceptable(&heartbeat_env.create_handler_ctx((meta.clone(), instruction))));
// Upgrade region
let instruction = Instruction::UpgradeRegion(UpgradeRegion {
region_id,
last_entry_id: None,
wait_for_replay_timeout: None,
});
assert!(
heartbeat_handler.is_acceptable(&heartbeat_env.create_handler_ctx((meta, instruction)))
);
}
fn close_region_instruction(region_id: RegionId) -> Instruction {
Instruction::CloseRegion(RegionIdent {
table_id: region_id.table_id(),

View File

@@ -14,6 +14,7 @@
use common_error::ext::ErrorExt;
use common_meta::instruction::{InstructionReply, OpenRegion, SimpleReply};
use common_meta::wal::prepare_wal_option;
use futures_util::future::BoxFuture;
use store_api::path_utils::region_dir;
use store_api::region_request::{RegionOpenRequest, RegionRequest};
@@ -26,15 +27,14 @@ impl HandlerContext {
OpenRegion {
region_ident,
region_storage_path,
region_options,
mut region_options,
region_wal_options,
skip_wal_replay,
}: OpenRegion,
) -> BoxFuture<'static, InstructionReply> {
Box::pin(async move {
let region_id = Self::region_ident_to_region_id(&region_ident);
// TODO(niebayes): extends region options with region_wal_options.
let _ = region_wal_options;
prepare_wal_option(&mut region_options, region_id, &region_wal_options);
let request = RegionRequest::Open(RegionOpenRequest {
engine: region_ident.engine,
region_dir: region_dir(&region_storage_path, region_id),
@@ -42,10 +42,8 @@ impl HandlerContext {
skip_wal_replay,
});
let result = self.region_server.handle_request(region_id, request).await;
let success = result.is_ok();
let error = result.as_ref().map_err(|e| e.output_msg()).err();
InstructionReply::OpenRegion(SimpleReply {
result: success,
error,

View File

@@ -14,7 +14,7 @@
use common_error::ext::ErrorExt;
use common_meta::instruction::{InstructionReply, UpgradeRegion, UpgradeRegionReply};
use common_telemetry::warn;
use common_telemetry::{info, warn};
use futures_util::future::BoxFuture;
use store_api::region_request::{RegionCatchupRequest, RegionRequest};
@@ -56,6 +56,7 @@ impl HandlerContext {
.try_register(
region_id,
Box::pin(async move {
info!("Executing region: {region_id} catchup to: last entry id {last_entry_id:?}");
region_server_moved
.handle_request(
region_id,

View File

@@ -24,5 +24,5 @@ pub mod heartbeat;
pub mod metrics;
pub mod region_server;
mod store;
#[cfg(test)]
mod tests;
#[cfg(any(test, feature = "testing"))]
pub mod tests;

View File

@@ -24,26 +24,26 @@ pub const REGION_ID: &str = "region_id";
lazy_static! {
/// The elapsed time of handling a request in the region_server.
pub static ref HANDLE_REGION_REQUEST_ELAPSED: HistogramVec = register_histogram_vec!(
"datanode_handle_region_request_elapsed",
"greptime_datanode_handle_region_request_elapsed",
"datanode handle region request elapsed",
&[REGION_REQUEST_TYPE]
)
.unwrap();
/// The elapsed time since the last received heartbeat.
pub static ref LAST_RECEIVED_HEARTBEAT_ELAPSED: IntGauge = register_int_gauge!(
"last_received_heartbeat_lease_elapsed",
"greptime_last_received_heartbeat_lease_elapsed",
"last received heartbeat lease elapsed",
)
.unwrap();
pub static ref LEASE_EXPIRED_REGION: IntGaugeVec = register_int_gauge_vec!(
"lease_expired_region",
"greptime_lease_expired_region",
"lease expired region",
&[REGION_ID]
)
.unwrap();
/// The received region leases via heartbeat.
pub static ref HEARTBEAT_REGION_LEASES: IntGaugeVec = register_int_gauge_vec!(
"heartbeat_region_leases",
"greptime_heartbeat_region_leases",
"received region leases via heartbeat",
&[REGION_ROLE]
)

View File

@@ -43,6 +43,7 @@ use datafusion_common::DataFusionError;
use datafusion_expr::{Expr as DfExpr, TableProviderFilterPushDown, TableType};
use datatypes::arrow::datatypes::SchemaRef;
use futures_util::future::try_join_all;
use metric_engine::engine::MetricEngine;
use prost::Message;
use query::QueryEngineRef;
use servers::error::{self as servers_error, ExecuteGrpcRequestSnafu, Result as ServerResult};
@@ -51,6 +52,7 @@ use servers::grpc::region_server::RegionServerHandler;
use session::context::{QueryContextBuilder, QueryContextRef};
use snafu::{OptionExt, ResultExt};
use store_api::metadata::RegionMetadataRef;
use store_api::metric_engine_consts::{METRIC_ENGINE_NAME, PHYSICAL_TABLE_METADATA_KEY};
use store_api::region_engine::{RegionEngineRef, RegionRole, SetReadonlyResponse};
use store_api::region_request::{AffectedRows, RegionCloseRequest, RegionRequest};
use store_api::storage::{RegionId, ScanRequest};
@@ -60,8 +62,9 @@ use tonic::{Request, Response, Result as TonicResult};
use crate::error::{
self, BuildRegionRequestsSnafu, DecodeLogicalPlanSnafu, ExecuteLogicalPlanSnafu,
GetRegionMetadataSnafu, HandleRegionRequestSnafu, RegionEngineNotFoundSnafu,
RegionNotFoundSnafu, Result, StopRegionEngineSnafu, UnsupportedOutputSnafu,
FindLogicalRegionsSnafu, GetRegionMetadataSnafu, HandleRegionRequestSnafu,
RegionEngineNotFoundSnafu, RegionNotFoundSnafu, Result, StopRegionEngineSnafu, UnexpectedSnafu,
UnsupportedOutputSnafu,
};
use crate::event_listener::RegionServerEventListenerRef;
@@ -123,7 +126,10 @@ impl RegionServer {
self.inner.handle_read(request).await
}
pub fn opened_regions(&self) -> Vec<RegionStat> {
/// Returns all opened and reportable regions.
///
/// Notes: except all metrics regions.
pub fn reportable_regions(&self) -> Vec<RegionStat> {
self.inner
.region_map
.iter()
@@ -369,7 +375,7 @@ impl RegionServerInner {
let current_region_status = self.region_map.get(&region_id);
let engine = match region_change {
RegionChange::Register(ref engine_type) => match current_region_status {
RegionChange::Register(ref engine_type, _) => match current_region_status {
Some(status) => match status.clone() {
RegionEngineWithStatus::Registering(_) => {
return Ok(CurrentEngine::EarlyReturn(0))
@@ -427,8 +433,12 @@ impl RegionServerInner {
.start_timer();
let region_change = match &request {
RegionRequest::Create(create) => RegionChange::Register(create.engine.clone()),
RegionRequest::Open(open) => RegionChange::Register(open.engine.clone()),
RegionRequest::Create(create) => RegionChange::Register(create.engine.clone(), false),
RegionRequest::Open(open) => {
let is_opening_physical_region =
open.options.contains_key(PHYSICAL_TABLE_METADATA_KEY);
RegionChange::Register(open.engine.clone(), is_opening_physical_region)
}
RegionRequest::Close(_) | RegionRequest::Drop(_) => RegionChange::Deregisters,
RegionRequest::Put(_)
| RegionRequest::Delete(_)
@@ -460,7 +470,8 @@ impl RegionServerInner {
{
Ok(result) => {
// Sets corresponding region status to ready.
self.set_region_status_ready(region_id, engine, region_change);
self.set_region_status_ready(region_id, engine, region_change)
.await?;
Ok(result)
}
Err(err) => {
@@ -478,7 +489,7 @@ impl RegionServerInner {
region_change: &RegionChange,
) {
match region_change {
RegionChange::Register(_) => {
RegionChange::Register(_, _) => {
self.region_map.insert(
region_id,
RegionEngineWithStatus::Registering(engine.clone()),
@@ -497,7 +508,7 @@ impl RegionServerInner {
fn unset_region_status(&self, region_id: RegionId, region_change: RegionChange) {
match region_change {
RegionChange::None => {}
RegionChange::Register(_) | RegionChange::Deregisters => {
RegionChange::Register(_, _) | RegionChange::Deregisters => {
self.region_map
.remove(&region_id)
.map(|(id, engine)| engine.set_writable(id, false));
@@ -505,16 +516,20 @@ impl RegionServerInner {
}
}
fn set_region_status_ready(
async fn set_region_status_ready(
&self,
region_id: RegionId,
engine: RegionEngineRef,
region_change: RegionChange,
) {
) -> Result<()> {
let engine_type = engine.name();
match region_change {
RegionChange::None => {}
RegionChange::Register(_) => {
RegionChange::Register(_, is_opening_physical_region) => {
if is_opening_physical_region {
self.register_logical_regions(&engine, region_id).await?;
}
info!("Region {region_id} is registered to engine {engine_type}");
self.region_map
.insert(region_id, RegionEngineWithStatus::Ready(engine));
@@ -528,6 +543,37 @@ impl RegionServerInner {
self.event_listener.on_region_deregistered(region_id);
}
}
Ok(())
}
async fn register_logical_regions(
&self,
engine: &RegionEngineRef,
physical_region_id: RegionId,
) -> Result<()> {
let metric_engine =
engine
.as_any()
.downcast_ref::<MetricEngine>()
.context(UnexpectedSnafu {
violated: format!(
"expecting engine type '{}', actual '{}'",
METRIC_ENGINE_NAME,
engine.name(),
),
})?;
let logical_regions = metric_engine
.logical_regions(physical_region_id)
.await
.context(FindLogicalRegionsSnafu { physical_region_id })?;
for region in logical_regions {
self.region_map
.insert(region, RegionEngineWithStatus::Ready(engine.clone()));
info!("Logical region {} is registered!", region);
}
Ok(())
}
pub async fn handle_read(&self, request: QueryRequest) -> Result<SendableRecordBatchStream> {
@@ -622,7 +668,7 @@ impl RegionServerInner {
enum RegionChange {
None,
Register(String),
Register(String, bool),
Deregisters,
}
@@ -1051,7 +1097,7 @@ mod tests {
CurrentEngineTest {
region_id,
current_region_status: None,
region_change: RegionChange::Register(engine.name().to_string()),
region_change: RegionChange::Register(engine.name().to_string(), false),
assert: Box::new(|result| {
let current_engine = result.unwrap();
assert_matches!(current_engine, CurrentEngine::Engine(_));
@@ -1060,7 +1106,7 @@ mod tests {
CurrentEngineTest {
region_id,
current_region_status: Some(RegionEngineWithStatus::Registering(engine.clone())),
region_change: RegionChange::Register(engine.name().to_string()),
region_change: RegionChange::Register(engine.name().to_string(), false),
assert: Box::new(|result| {
let current_engine = result.unwrap();
assert_matches!(current_engine, CurrentEngine::EarlyReturn(_));
@@ -1069,7 +1115,7 @@ mod tests {
CurrentEngineTest {
region_id,
current_region_status: Some(RegionEngineWithStatus::Deregistering(engine.clone())),
region_change: RegionChange::Register(engine.name().to_string()),
region_change: RegionChange::Register(engine.name().to_string(), false),
assert: Box::new(|result| {
let err = result.unwrap_err();
assert_eq!(err.status_code(), StatusCode::RegionBusy);
@@ -1078,7 +1124,7 @@ mod tests {
CurrentEngineTest {
region_id,
current_region_status: Some(RegionEngineWithStatus::Ready(engine.clone())),
region_change: RegionChange::Register(engine.name().to_string()),
region_change: RegionChange::Register(engine.name().to_string(), false),
assert: Box::new(|result| {
let current_engine = result.unwrap();
assert_matches!(current_engine, CurrentEngine::Engine(_));

View File

@@ -26,10 +26,10 @@ use std::{env, path};
use common_base::readable_size::ReadableSize;
use common_telemetry::logging::info;
use object_store::layers::{LoggingLayer, LruCacheLayer, RetryLayer, TracingLayer};
use object_store::services::Fs as FsBuilder;
use object_store::util::normalize_dir;
use object_store::{util, HttpClient, ObjectStore, ObjectStoreBuilder};
use object_store::layers::{LruCacheLayer, RetryLayer};
use object_store::services::Fs;
use object_store::util::{join_dir, normalize_dir, with_instrument_layers};
use object_store::{HttpClient, ObjectStore, ObjectStoreBuilder};
use snafu::prelude::*;
use crate::config::{ObjectStoreConfig, DEFAULT_OBJECT_STORE_CACHE_SIZE};
@@ -60,16 +60,7 @@ pub(crate) async fn new_object_store(
object_store
};
let store = object_store
.layer(
LoggingLayer::default()
// Print the expected error only in DEBUG level.
// See https://docs.rs/opendal/latest/opendal/layers/struct.LoggingLayer.html#method.with_error_level
.with_error_level(Some("debug"))
.expect("input error level must be valid"),
)
.layer(TracingLayer)
.layer(object_store::layers::PrometheusMetricsLayer);
let store = with_instrument_layers(object_store);
Ok(store)
}
@@ -114,11 +105,10 @@ async fn create_object_store_with_cache(
};
if let Some(path) = cache_path {
let path = util::normalize_dir(path);
let atomic_temp_dir = format!("{path}.tmp/");
let atomic_temp_dir = join_dir(path, ".tmp/");
clean_temp_dir(&atomic_temp_dir)?;
let cache_store = FsBuilder::default()
.root(&path)
let cache_store = Fs::default()
.root(path)
.atomic_write_dir(&atomic_temp_dir)
.build()
.context(error::InitBackendSnafu)?;

View File

@@ -13,7 +13,7 @@
// limitations under the License.
use common_telemetry::logging::info;
use object_store::services::Azblob as AzureBuilder;
use object_store::services::Azblob;
use object_store::{util, ObjectStore};
use secrecy::ExposeSecret;
use snafu::prelude::*;
@@ -30,7 +30,7 @@ pub(crate) async fn new_azblob_object_store(azblob_config: &AzblobConfig) -> Res
azblob_config.container, &root
);
let mut builder = AzureBuilder::default();
let mut builder = Azblob::default();
let _ = builder
.root(&root)
.container(&azblob_config.container)

View File

@@ -15,7 +15,8 @@
use std::{fs, path};
use common_telemetry::logging::info;
use object_store::services::Fs as FsBuilder;
use object_store::services::Fs;
use object_store::util::join_dir;
use object_store::ObjectStore;
use snafu::prelude::*;
@@ -31,10 +32,10 @@ pub(crate) async fn new_fs_object_store(
.context(error::CreateDirSnafu { dir: data_home })?;
info!("The file storage home is: {}", data_home);
let atomic_write_dir = format!("{data_home}.tmp/");
let atomic_write_dir = join_dir(data_home, ".tmp/");
store::clean_temp_dir(&atomic_write_dir)?;
let mut builder = FsBuilder::default();
let mut builder = Fs::default();
let _ = builder.root(data_home).atomic_write_dir(&atomic_write_dir);
let object_store = ObjectStore::new(builder)

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