Compare commits

...

35 Commits

Author SHA1 Message Date
luofucong
517e9362ad build: release v0.3.0 2023-06-05 10:38:51 +08:00
Yingwen
466f258266 feat(servers): collect samples by metric (#1706) 2023-06-03 17:17:52 +08:00
Ruihang Xia
94228285a7 feat: convert values to vector directly (#1704)
Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2023-06-03 12:41:13 +08:00
JeremyHi
3d7185749d feat: insert with stream (#1703)
* feat: insert with stream

* chore: by CR
2023-06-03 03:58:00 +00:00
LFC
5004cf6d9a feat: make grpc insert requests in a batch (#1687)
* feat: make Prometheus remote write in a batch

* rebase

* fix: resolve PR comments

* fix: resolve PR comments

* fix: resolve PR comments
2023-06-02 09:06:48 +00:00
Ruihang Xia
8e69aef973 feat: serialize/deserialize support for PromQL plans (#1684)
* implement serializer

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

* fix clippy and CR comments

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

* fix compile error

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

* register registry

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

* enable promql plan for dist planner

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

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2023-06-02 16:14:05 +08:00
Ruihang Xia
2615718999 feat: merge scan for distributed execution (#1660)
* generate exec plan

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

* move DatanodeClients to client crate

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

* wip MergeScanExec::to_stream

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

* fix compile errors

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

* fix default catalog

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

* fix expand order of new stage

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

* move sqlness cases contains plan out of common dir

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

* refactor information schema to allow duplicated scan call

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

* fix: ignore two cases due to substrait

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

* reorganise sqlness common cases

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

* fix typos

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

* redact round robin partition number

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

* Apply suggestions from code review

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

* skip tranforming projection

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

* update sqlness result

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

* revert common/order

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

* fix clippy

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

* Update src/query/src/dist_plan/merge_scan.rs

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

* update sqlness result

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

* update sqlness result again

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

* resolve CR comments

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

* ignore region failover IT

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

* update sqlness result again and again

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

* unignore some tests about projection

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

* enable failover tests

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

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
Co-authored-by: Yingwen <realevenyag@gmail.com>
Co-authored-by: JeremyHi <jiachun_feng@proton.me>
2023-06-02 06:42:54 +00:00
fys
fe6e3daf81 fix: failed to insert data with u8 (#1701)
* fix: failed to insert data with u8 field

* remove unused code

* fix cr
2023-06-02 06:01:59 +00:00
ZonaHe
b7e1778ada feat: update dashboard to v0.2.6 (#1700)
Co-authored-by: ZonaHex <ZonaHex@users.noreply.github.com>
2023-06-02 13:26:07 +08:00
Lei, HUANG
ccd666aa9b fix: avoid writing manifest and wal if no files are actually flushed (#1698)
* fix: avoid writing manifest and wal if no files are actually flushed

* fix: simplify log
2023-06-02 13:16:59 +08:00
JeremyHi
2aa442c86d feat: exists API for KVStore (#1695)
* feat: exists API for kv

* chore: add unit test
2023-06-02 12:35:04 +08:00
Weny Xu
f811ae4665 fix: enable region failover test (#1699)
fix: fix region failover test
2023-06-02 12:05:37 +08:00
Ruihang Xia
e5b6f8654a feat: optimizer rule to pass expected output ordering hint (#1675)
* move type convertsion rule into optimizer dir

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

* implement order_hint rule

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

* add test

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

* fix clippy

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

* it works!

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

* fix clippy

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

* use column name instead

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

* fix clippy

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

* accomplish test case

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

* update lock file

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

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2023-06-02 03:43:51 +00:00
Ruihang Xia
ff6d11ddc7 chore: ignore symbol link target file (#1696)
Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2023-06-02 10:42:44 +08:00
Ruihang Xia
878c6bf75a fix: do not alias relation before join (#1693)
* fix: do not alias relation before join

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

* Update src/promql/src/error.rs

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

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
Co-authored-by: Lei, HUANG <6406592+v0y4g3r@users.noreply.github.com>
Co-authored-by: dennis zhuang <killme2008@gmail.com>
2023-06-01 14:24:37 +00:00
LFC
ce440606a9 fix: sqlness failed due to region failover wrongly kicks in for dropp… (#1690)
fix: sqlness failed due to region failover wrongly kicks in for dropped or renamed table
2023-06-01 21:47:47 +08:00
fys
5fd7250dca fix: invalidate route cache on renaming table (#1691)
* fix: sqlness test

* remove unnecessary clone

* fix cr
2023-06-01 20:43:31 +08:00
Ruihang Xia
5a5e88353c fix: do not change timestamp index column while planning aggr (#1688)
* fix: do not change timestamp index column while planning aggr

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

* remove println

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

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2023-06-01 20:17:18 +08:00
Ruihang Xia
ef15de5f17 ci: always upload sqlness log (#1692)
Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2023-06-01 20:01:26 +08:00
fys
86adac1532 chore: reject table creation when partitions exceeds peer number (#1654)
* chore: table creation is rejected, when partition_num exceeds peer_num

* chore: modify no_active_datanode error msg

* fix: ut

* fix sqlness test and add limit for select peer in region_failover

* upgrade greptime-proto

* self cr

* fix: cargo sqlness

* chore: add table info in select ctx for failover

* fix sqlness
2023-06-01 09:05:17 +00:00
Ning Sun
e7a410573b test: fix sqlx compatibility and adds integration test for sqlx (#1686)
* test: fix sqlx compatibility and adds integration test for sqlx

* test: correct insert statements
2023-06-01 15:43:13 +08:00
Yingwen
548f0d1e2a feat: Add app version metric (#1685)
* feat: Add app version metric

* chore: use greptimedb instead of greptime
2023-06-01 14:31:08 +08:00
Zheming Li
5467ea496f feat: Add column supports at first or after the existing columns (#1621)
* feat: Add column supports at first or after the existing columns

* Update src/common/query/Cargo.toml

---------

Co-authored-by: dennis zhuang <killme2008@gmail.com>
2023-06-01 02:13:00 +00:00
Yingwen
70e17ead68 fix: Print source error in subprocedure failure message (#1683)
* fix: print source error in subprocedure failed error

* feat: print source error in subprocedure failure message
2023-06-01 09:51:31 +08:00
dennis zhuang
ae8203fafa fix: prepare statement doesn't support insert clause (#1680)
* fix: insert clause doesn't support prepare statement

* fix: manifeste dir

* fix: format

* fix: temp path
2023-05-31 20:14:58 +08:00
Ruihang Xia
ac3666b841 chore(deps): bump arrow/parquet to 40.0, datafuson to the latest HEAD (#1677)
* fix compile error

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

* remove deprecated substrait

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

* fix clippy

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

* update deps

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

* downgrade opendal to 0.33.1

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

* change finish's impl

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

* update test results

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

* ignore failing cases

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

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2023-05-31 18:55:02 +08:00
Weny Xu
0460f3ae30 test: add write test for region failover (#1673)
* test: add write test for region failover

* test: add written data assertion after failover

* test: support more storage types
2023-05-31 15:42:00 +08:00
Yingwen
9d179802b8 feat: Add a global TTL option for all tables (#1679)
* feat: Add a global TTL option for all tables

* docs: update config examples

* chore: print start command and options when standalone/frontend starts
2023-05-31 15:36:25 +08:00
Lei, HUANG
72b6bd11f7 feat: adapt window reader to order rules (#1671)
* feat: adapt window reader to order rules

* fix: add asc sort test case
2023-05-31 03:36:17 +00:00
Xuanwo
6b08a5f94e chore: Bump OpenDAL to v0.36 (#1678)
* chore: Bump OpenDAL to v0.36

Signed-off-by: Xuanwo <github@xuanwo.io>

* Fix

Signed-off-by: Xuanwo <github@xuanwo.io>

---------

Signed-off-by: Xuanwo <github@xuanwo.io>
2023-05-31 11:12:40 +08:00
dennis zhuang
00104bef76 feat: supports CTE query (#1674)
* feat: supports CTE query

* test: move cte test to standalone
2023-05-30 12:08:49 +00:00
Zou Wei
ae81c7329d feat: support azblob storage. (#1659)
* feat:support azblob storage.

* test:add some tests.

* refactor:use if-let.
2023-05-30 19:59:38 +08:00
Yingwen
c5f6d7c99a refactor: update proto and rename incorrect region_id fields (#1670) 2023-05-30 15:19:04 +09:00
Weny Xu
bb1b71bcf0 feat: acquire table_id from region_id (#1656)
feat: acquire table_id from region_id
2023-05-30 03:36:47 +00:00
Weny Xu
a4b884406a feat: add invalidate cache step (#1658)
* feat: add invalidate cache step

* refactor: refactor TableIdent

* chore: apply suggestions from CR
2023-05-30 11:17:59 +08:00
264 changed files with 8186 additions and 4319 deletions

View File

@@ -9,3 +9,9 @@ GT_OSS_BUCKET=OSS bucket
GT_OSS_ACCESS_KEY_ID=OSS access key id
GT_OSS_ACCESS_KEY=OSS access key
GT_OSS_ENDPOINT=OSS endpoint
# Settings for azblob test
GT_AZBLOB_CONTAINER=AZBLOB container
GT_AZBLOB_ACCOUNT_NAME=AZBLOB account name
GT_AZBLOB_ACCOUNT_KEY=AZBLOB account key
GT_AZBLOB_ENDPOINT=AZBLOB endpoint

View File

@@ -141,6 +141,7 @@ jobs:
- name: Run sqlness
run: cargo sqlness && ls /tmp
- name: Upload sqlness logs
if: always()
uses: actions/upload-artifact@v3
with:
name: sqlness-logs

2
.gitignore vendored
View File

@@ -1,6 +1,8 @@
# Generated by Cargo
# will have compiled files and executables
/target/
# also ignore if it's a symbolic link
/target
# Remove Cargo.lock from gitignore if creating an executable, leave it for libraries
# More information here https://doc.rust-lang.org/cargo/guide/cargo-toml-vs-cargo-lock.html

1690
Cargo.lock generated

File diff suppressed because it is too large Load Diff

View File

@@ -49,36 +49,38 @@ members = [
]
[workspace.package]
version = "0.2.0"
version = "0.3.0"
edition = "2021"
license = "Apache-2.0"
[workspace.dependencies]
arrow = { version = "37.0" }
arrow-array = "37.0"
arrow-flight = "37.0"
arrow-schema = { version = "37.0", features = ["serde"] }
arrow = { version = "40.0" }
arrow-array = "40.0"
arrow-flight = "40.0"
arrow-schema = { version = "40.0", features = ["serde"] }
async-stream = "0.3"
async-trait = "0.1"
chrono = { version = "0.4", features = ["serde"] }
# TODO(ruihang): use arrow-datafusion when it contains https://github.com/apache/arrow-datafusion/pull/6032
datafusion = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "5337c86120de8193406b59be7612484796a46294" }
datafusion-common = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "5337c86120de8193406b59be7612484796a46294" }
datafusion-expr = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "5337c86120de8193406b59be7612484796a46294" }
datafusion-optimizer = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "5337c86120de8193406b59be7612484796a46294" }
datafusion-physical-expr = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "5337c86120de8193406b59be7612484796a46294" }
datafusion-sql = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "5337c86120de8193406b59be7612484796a46294" }
datafusion-substrait = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "5337c86120de8193406b59be7612484796a46294" }
datafusion = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "63e52dde9e44cac4b1f6c6e6b6bf6368ba3bd323" }
datafusion-common = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "63e52dde9e44cac4b1f6c6e6b6bf6368ba3bd323" }
datafusion-expr = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "63e52dde9e44cac4b1f6c6e6b6bf6368ba3bd323" }
datafusion-optimizer = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "63e52dde9e44cac4b1f6c6e6b6bf6368ba3bd323" }
datafusion-physical-expr = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "63e52dde9e44cac4b1f6c6e6b6bf6368ba3bd323" }
datafusion-sql = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "63e52dde9e44cac4b1f6c6e6b6bf6368ba3bd323" }
datafusion-substrait = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "63e52dde9e44cac4b1f6c6e6b6bf6368ba3bd323" }
futures = "0.3"
futures-util = "0.3"
parquet = "37.0"
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "4398d20c56d5f7939cc2960789cb1fa7dd18e6fe" }
itertools = "0.10"
parquet = "40.0"
paste = "1.0"
prost = "0.11"
rand = "0.8"
serde = { version = "1.0", features = ["derive"] }
serde_json = "1.0"
snafu = { version = "0.7", features = ["backtraces"] }
sqlparser = "0.33"
sqlparser = "0.34"
tempfile = "3"
tokio = { version = "1.28", features = ["full"] }
tokio-util = { version = "0.7", features = ["io-util", "compat"] }

View File

@@ -9,6 +9,6 @@ arrow.workspace = true
clap = { version = "4.0", features = ["derive"] }
client = { path = "../src/client" }
indicatif = "0.17.1"
itertools = "0.10.5"
itertools.workspace = true
parquet.workspace = true
tokio.workspace = true

View File

@@ -26,7 +26,9 @@ use arrow::datatypes::{DataType, Float64Type, Int64Type};
use arrow::record_batch::RecordBatch;
use clap::Parser;
use client::api::v1::column::Values;
use client::api::v1::{Column, ColumnDataType, ColumnDef, CreateTableExpr, InsertRequest};
use client::api::v1::{
Column, ColumnDataType, ColumnDef, CreateTableExpr, InsertRequest, InsertRequests,
};
use client::{Client, Database, DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
use indicatif::{MultiProgress, ProgressBar, ProgressStyle};
use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder;
@@ -107,8 +109,12 @@ async fn write_data(
columns,
row_count,
};
let requests = InsertRequests {
inserts: vec![request],
};
let now = Instant::now();
db.insert(request).await.unwrap();
db.insert(requests).await.unwrap();
let elapsed = now.elapsed();
total_rpc_elapsed_ms += elapsed.as_millis();
progress_bar.inc(row_count as _);
@@ -364,7 +370,7 @@ fn create_table_expr() -> CreateTableExpr {
primary_keys: vec!["VendorID".to_string()],
create_if_not_exists: false,
table_options: Default::default(),
region_ids: vec![0],
region_numbers: vec![0],
table_id: None,
engine: "mito".to_string(),
}

View File

@@ -36,6 +36,8 @@ sync_write = false
[storage]
type = "File"
data_home = "/tmp/greptimedb/"
# TTL for all tables. Disabled by default.
# global_ttl = "7d"
# Compaction options, see `standalone.example.toml`.
[storage.compaction]
@@ -71,7 +73,9 @@ global_write_buffer_size = "1GB"
max_retry_times = 3
retry_delay = "500ms"
# Log options, see `standalone.example.toml`
[logging]
dir = "/tmp/greptimedb/logs"
level = "info"
# Log options
# [logging]
# Specify logs directory.
# dir = "/tmp/greptimedb/logs"
# Specify the log level [info | debug | error | warn]
# level = "info"

View File

@@ -58,6 +58,6 @@ connect_timeout_millis = 5000
tcp_nodelay = true
# Log options, see `standalone.example.toml`
[logging]
dir = "/tmp/greptimedb/logs"
level = "info"
# [logging]
# dir = "/tmp/greptimedb/logs"
# level = "info"

View File

@@ -15,6 +15,6 @@ selector = "LeaseBased"
use_memory_store = false
# Log options, see `standalone.example.toml`
[logging]
dir = "/tmp/greptimedb/logs"
level = "info"
# [logging]
# dir = "/tmp/greptimedb/logs"
# level = "info"

View File

@@ -97,6 +97,8 @@ sync_write = false
type = "File"
# Data directory, "/tmp/greptimedb/data" by default.
data_home = "/tmp/greptimedb/"
# TTL for all tables. Disabled by default.
# global_ttl = "7d"
# Compaction options.
[storage.compaction]
@@ -138,8 +140,8 @@ max_retry_times = 3
retry_delay = "500ms"
# Log options
[logging]
# [logging]
# Specify logs directory.
dir = "/tmp/greptimedb/logs"
# dir = "/tmp/greptimedb/logs"
# Specify the log level [info | debug | error | warn]
level = "debug"
# level = "info"

View File

@@ -10,7 +10,7 @@ common-base = { path = "../common/base" }
common-error = { path = "../common/error" }
common-time = { path = "../common/time" }
datatypes = { path = "../datatypes" }
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "f43972af48f8be1cbb1d27640a8aba8b30955028" }
greptime-proto.workspace = true
prost.workspace = true
snafu = { version = "0.7", features = ["backtraces"] }
tonic.workspace = true

View File

@@ -231,7 +231,7 @@ pub fn push_vals(column: &mut Column, origin_count: usize, vector: VectorRef) {
/// Returns the type name of the [Request].
pub fn request_type(request: &Request) -> &'static str {
match request {
Request::Insert(_) => "insert",
Request::Inserts(_) => "inserts",
Request::Query(query_req) => query_request_type(query_req),
Request::Ddl(ddl_req) => ddl_request_type(ddl_req),
Request::Delete(_) => "delete",

View File

@@ -23,4 +23,5 @@ pub mod prometheus {
pub mod v1;
pub use greptime_proto;
pub use prost::DecodeError;

View File

@@ -4,6 +4,9 @@ version.workspace = true
edition.workspace = true
license.workspace = true
[features]
testing = []
[dependencies]
api = { path = "../api" }
arc-swap = "1.0"
@@ -42,6 +45,7 @@ table = { path = "../table" }
tokio.workspace = true
[dev-dependencies]
catalog = { path = ".", features = ["testing"] }
common-test-util = { path = "../common/test-util" }
chrono.workspace = true
log-store = { path = "../log-store" }

View File

@@ -16,17 +16,18 @@ mod columns;
mod tables;
use std::any::Any;
use std::sync::{Arc, Mutex};
use std::sync::Arc;
use async_trait::async_trait;
use common_error::prelude::BoxedError;
use common_query::physical_plan::PhysicalPlanRef;
use common_query::prelude::Expr;
use common_recordbatch::{RecordBatchStreamAdaptor, SendableRecordBatchStream};
use datatypes::schema::SchemaRef;
use futures_util::StreamExt;
use snafu::{OptionExt, ResultExt};
use snafu::ResultExt;
use store_api::storage::ScanRequest;
use table::error::{DuplicatedExecuteCallSnafu, SchemaConversionSnafu};
use table::error::{SchemaConversionSnafu, TablesRecordBatchSnafu};
use table::{Result as TableResult, Table, TableRef};
use self::columns::InformationSchemaColumns;
@@ -64,23 +65,21 @@ impl SchemaProvider for InformationSchemaProvider {
}
async fn table(&self, name: &str) -> Result<Option<TableRef>> {
let stream = match name.to_ascii_lowercase().as_ref() {
TABLES => InformationSchemaTables::new(
let stream_builder = match name.to_ascii_lowercase().as_ref() {
TABLES => Arc::new(InformationSchemaTables::new(
self.catalog_name.clone(),
self.catalog_provider.clone(),
)
.to_stream()?,
COLUMNS => InformationSchemaColumns::new(
)) as _,
COLUMNS => Arc::new(InformationSchemaColumns::new(
self.catalog_name.clone(),
self.catalog_provider.clone(),
)
.to_stream()?,
)) as _,
_ => {
return Ok(None);
}
};
Ok(Some(Arc::new(InformationTable::new(stream))))
Ok(Some(Arc::new(InformationTable::new(stream_builder))))
}
async fn table_exist(&self, name: &str) -> Result<bool> {
@@ -89,18 +88,21 @@ impl SchemaProvider for InformationSchemaProvider {
}
}
// TODO(ruihang): make it a more generic trait:
// https://github.com/GreptimeTeam/greptimedb/pull/1639#discussion_r1205001903
pub trait InformationStreamBuilder: Send + Sync {
fn to_stream(&self) -> Result<SendableRecordBatchStream>;
fn schema(&self) -> SchemaRef;
}
pub struct InformationTable {
schema: SchemaRef,
stream: Arc<Mutex<Option<SendableRecordBatchStream>>>,
stream_builder: Arc<dyn InformationStreamBuilder>,
}
impl InformationTable {
pub fn new(stream: SendableRecordBatchStream) -> Self {
let schema = stream.schema();
Self {
schema,
stream: Arc::new(Mutex::new(Some(stream))),
}
pub fn new(stream_builder: Arc<dyn InformationStreamBuilder>) -> Self {
Self { stream_builder }
}
}
@@ -111,7 +113,7 @@ impl Table for InformationTable {
}
fn schema(&self) -> SchemaRef {
self.schema.clone()
self.stream_builder.schema()
}
fn table_info(&self) -> table::metadata::TableInfoRef {
@@ -144,13 +146,10 @@ impl Table for InformationTable {
self.schema()
};
let stream = self
.stream
.lock()
.unwrap()
.take()
.with_context(|| DuplicatedExecuteCallSnafu {
table: self.table_info().name.clone(),
})?
.stream_builder
.to_stream()
.map_err(BoxedError::new)
.context(TablesRecordBatchSnafu)?
.map(move |batch| {
batch.and_then(|batch| {
if let Some(projection) = &projection {
@@ -163,6 +162,7 @@ impl Table for InformationTable {
let stream = RecordBatchStreamAdaptor {
schema: projected_schema,
stream: Box::pin(stream),
output_ordering: None,
};
Ok(Box::pin(stream))
}

View File

@@ -31,6 +31,7 @@ use datatypes::schema::{ColumnSchema, Schema, SchemaRef};
use datatypes::vectors::{StringVectorBuilder, VectorRef};
use snafu::ResultExt;
use super::InformationStreamBuilder;
use crate::error::{CreateRecordBatchSnafu, InternalSnafu, Result};
use crate::CatalogProviderRef;
@@ -71,9 +72,15 @@ impl InformationSchemaColumns {
self.catalog_provider.clone(),
)
}
}
pub fn to_stream(&self) -> Result<SendableRecordBatchStream> {
let schema = self.schema().clone();
impl InformationStreamBuilder for InformationSchemaColumns {
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
fn to_stream(&self) -> Result<SendableRecordBatchStream> {
let schema = self.schema.arrow_schema().clone();
let mut builder = self.builder();
let stream = Box::pin(DfRecordBatchStreamAdapter::new(
schema,
@@ -190,7 +197,7 @@ impl DfPartitionStream for InformationSchemaColumns {
}
fn execute(&self, _: Arc<TaskContext>) -> DfSendableRecordBatchStream {
let schema = self.schema().clone();
let schema = self.schema.arrow_schema().clone();
let mut builder = self.builder();
Box::pin(DfRecordBatchStreamAdapter::new(
schema,

View File

@@ -30,6 +30,7 @@ use snafu::ResultExt;
use table::metadata::TableType;
use crate::error::{CreateRecordBatchSnafu, InternalSnafu, Result};
use crate::information_schema::InformationStreamBuilder;
use crate::CatalogProviderRef;
pub(super) struct InformationSchemaTables {
@@ -62,9 +63,15 @@ impl InformationSchemaTables {
self.catalog_provider.clone(),
)
}
}
pub fn to_stream(&self) -> Result<SendableRecordBatchStream> {
let schema = self.schema().clone();
impl InformationStreamBuilder for InformationSchemaTables {
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
fn to_stream(&self) -> Result<SendableRecordBatchStream> {
let schema = self.schema.arrow_schema().clone();
let mut builder = self.builder();
let stream = Box::pin(DfRecordBatchStreamAdapter::new(
schema,
@@ -182,7 +189,7 @@ impl DfPartitionStream for InformationSchemaTables {
}
fn execute(&self, _: Arc<TaskContext>) -> DfSendableRecordBatchStream {
let schema = self.schema().clone();
let schema = self.schema.arrow_schema().clone();
let mut builder = self.builder();
Box::pin(DfRecordBatchStreamAdapter::new(
schema,

View File

@@ -12,6 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::any::Any;
use std::fmt::Debug;
use std::pin::Pin;
use std::sync::Arc;
@@ -26,6 +27,9 @@ use crate::error::Error;
mod client;
mod manager;
#[cfg(feature = "testing")]
pub mod mock;
#[derive(Debug, Clone)]
pub struct Kv(pub Vec<u8>, pub Vec<u8>);
@@ -70,6 +74,11 @@ pub trait KvBackend: Send + Sync {
}
return Ok(None);
}
/// MoveValue atomically renames the key to the given updated key.
async fn move_value(&self, from_key: &[u8], to_key: &[u8]) -> Result<(), Error>;
fn as_any(&self) -> &dyn Any;
}
pub type KvBackendRef = Arc<dyn KvBackend>;
@@ -121,6 +130,14 @@ mod tests {
async fn delete_range(&self, _key: &[u8], _end: &[u8]) -> Result<(), Error> {
unimplemented!()
}
async fn move_value(&self, _from_key: &[u8], _to_key: &[u8]) -> Result<(), Error> {
unimplemented!()
}
fn as_any(&self) -> &dyn Any {
self
}
}
#[tokio::test]

View File

@@ -12,12 +12,15 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::any::Any;
use std::fmt::Debug;
use std::sync::Arc;
use std::time::Duration;
use async_stream::stream;
use common_meta::rpc::store::{CompareAndPutRequest, DeleteRangeRequest, PutRequest, RangeRequest};
use common_meta::rpc::store::{
CompareAndPutRequest, DeleteRangeRequest, MoveValueRequest, PutRequest, RangeRequest,
};
use common_telemetry::{info, timer};
use meta_client::client::MetaClient;
use moka::future::{Cache, CacheBuilder};
@@ -32,9 +35,10 @@ const CACHE_MAX_CAPACITY: u64 = 10000;
const CACHE_TTL_SECOND: u64 = 10 * 60;
const CACHE_TTI_SECOND: u64 = 5 * 60;
pub type CacheBackendRef = Arc<Cache<Vec<u8>, Option<Kv>>>;
pub struct CachedMetaKvBackend {
kv_backend: KvBackendRef,
cache: Arc<Cache<Vec<u8>, Option<Kv>>>,
cache: CacheBackendRef,
}
#[async_trait::async_trait]
@@ -98,6 +102,21 @@ impl KvBackend for CachedMetaKvBackend {
ret
}
async fn move_value(&self, from_key: &[u8], to_key: &[u8]) -> Result<()> {
let ret = self.kv_backend.move_value(from_key, to_key).await;
if ret.is_ok() {
self.invalidate_key(from_key).await;
self.invalidate_key(to_key).await;
}
ret
}
fn as_any(&self) -> &dyn Any {
self
}
}
#[async_trait::async_trait]
@@ -130,6 +149,10 @@ impl CachedMetaKvBackend {
Self { kv_backend, cache }
}
pub fn cache(&self) -> &CacheBackendRef {
&self.cache
}
}
#[derive(Debug)]
@@ -214,4 +237,14 @@ impl KvBackend for MetaKvBackend {
Ok(Err(response.take_prev_kv().map(|v| v.value().to_vec())))
}
}
async fn move_value(&self, from_key: &[u8], to_key: &[u8]) -> Result<()> {
let req = MoveValueRequest::new(from_key, to_key);
self.client.move_value(req).await.context(MetaSrvSnafu)?;
Ok(())
}
fn as_any(&self) -> &dyn Any {
self
}
}

View File

@@ -12,6 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::any::Any;
use std::collections::btree_map::Entry;
use std::collections::{BTreeMap, HashMap};
use std::fmt::{Display, Formatter};
@@ -19,9 +20,6 @@ use std::str::FromStr;
use std::sync::Arc;
use async_stream::stream;
use catalog::error::Error;
use catalog::helper::{CatalogKey, CatalogValue, SchemaKey, SchemaValue};
use catalog::remote::{Kv, KvBackend, ValueIter};
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
use common_recordbatch::RecordBatch;
use common_telemetry::logging::info;
@@ -36,6 +34,10 @@ use table::test_util::MemTable;
use table::TableRef;
use tokio::sync::RwLock;
use crate::error::Error;
use crate::helper::{CatalogKey, CatalogValue, SchemaKey, SchemaValue};
use crate::remote::{Kv, KvBackend, ValueIter};
pub struct MockKvBackend {
map: RwLock<BTreeMap<Vec<u8>, Vec<u8>>>,
}
@@ -151,6 +153,14 @@ impl KvBackend for MockKvBackend {
}
Ok(())
}
async fn move_value(&self, _from_key: &[u8], _to_key: &[u8]) -> Result<(), Error> {
unimplemented!()
}
fn as_any(&self) -> &dyn Any {
self
}
}
#[derive(Default)]

View File

@@ -14,8 +14,6 @@
#![feature(assert_matches)]
mod mock;
#[cfg(test)]
mod tests {
use std::assert_matches::assert_matches;
@@ -23,6 +21,7 @@ mod tests {
use std::sync::Arc;
use catalog::helper::{CatalogKey, CatalogValue, SchemaKey, SchemaValue};
use catalog::remote::mock::{MockKvBackend, MockTableEngine};
use catalog::remote::{
CachedMetaKvBackend, KvBackend, KvBackendRef, RemoteCatalogManager, RemoteCatalogProvider,
RemoteSchemaProvider,
@@ -35,8 +34,6 @@ mod tests {
use table::engine::{EngineContext, TableEngineRef};
use table::requests::CreateTableRequest;
use crate::mock::{MockKvBackend, MockTableEngine};
#[tokio::test]
async fn test_backend() {
common_telemetry::init_default_ut_logging();

View File

@@ -4,6 +4,9 @@ version.workspace = true
edition.workspace = true
license.workspace = true
[features]
testing = []
[dependencies]
api = { path = "../api" }
arrow-flight.workspace = true
@@ -16,21 +19,24 @@ common-grpc-expr = { path = "../common/grpc-expr" }
common-query = { path = "../common/query" }
common-recordbatch = { path = "../common/recordbatch" }
common-time = { path = "../common/time" }
common-meta = { path = "../common/meta" }
common-telemetry = { path = "../common/telemetry" }
datafusion.workspace = true
datatypes = { path = "../datatypes" }
enum_dispatch = "0.3"
futures-util.workspace = true
moka = { version = "0.9", features = ["future"] }
parking_lot = "0.12"
prost.workspace = true
rand.workspace = true
snafu.workspace = true
tokio-stream = { version = "0.1", features = ["net"] }
tokio.workspace = true
tonic.workspace = true
[dev-dependencies]
datanode = { path = "../datanode" }
substrait = { path = "../common/substrait" }
tokio.workspace = true
tracing = "0.1"
tracing-subscriber = { version = "0.3", features = ["env-filter"] }
prost.workspace = true

View File

@@ -63,7 +63,7 @@ async fn run() {
create_if_not_exists: false,
table_options: Default::default(),
table_id: Some(TableId { id: 1024 }),
region_ids: vec![0],
region_numbers: vec![0],
engine: MITO_ENGINE.to_string(),
};

View File

@@ -12,15 +12,17 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::fmt::{Debug, Formatter};
use std::sync::{Arc, Mutex};
use std::time::Duration;
use client::Client;
use common_grpc::channel_manager::ChannelManager;
use common_grpc::channel_manager::{ChannelConfig, ChannelManager};
use common_meta::peer::Peer;
use common_telemetry::info;
use moka::future::{Cache, CacheBuilder};
use crate::Client;
pub struct DatanodeClients {
channel_manager: ChannelManager,
clients: Cache<Peer, Client>,
@@ -29,8 +31,11 @@ pub struct DatanodeClients {
impl Default for DatanodeClients {
fn default() -> Self {
// TODO(LFC): Make this channel config configurable.
let config = ChannelConfig::new().timeout(Duration::from_secs(8));
Self {
channel_manager: ChannelManager::new(),
channel_manager: ChannelManager::with_config(config),
clients: CacheBuilder::new(1024)
.time_to_live(Duration::from_secs(30 * 60))
.time_to_idle(Duration::from_secs(5 * 60))
@@ -40,8 +45,16 @@ impl Default for DatanodeClients {
}
}
impl Debug for DatanodeClients {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
f.debug_struct("DatanodeClients")
.field("channel_manager", &self.channel_manager)
.finish()
}
}
impl DatanodeClients {
pub(crate) fn start(&self) {
pub fn start(&self) {
let mut started = self.started.lock().unwrap();
if *started {
return;
@@ -53,7 +66,7 @@ impl DatanodeClients {
*started = true;
}
pub(crate) async fn get_client(&self, datanode: &Peer) -> Client {
pub async fn get_client(&self, datanode: &Peer) -> Client {
self.clients
.get_with_by_ref(datanode, async move {
Client::with_manager_and_urls(

View File

@@ -18,7 +18,7 @@ use api::v1::greptime_request::Request;
use api::v1::query_request::Query;
use api::v1::{
greptime_response, AffectedRows, AlterExpr, AuthHeader, CreateTableExpr, DdlRequest,
DeleteRequest, DropTableExpr, FlushTableExpr, GreptimeRequest, InsertRequest, PromRangeQuery,
DeleteRequest, DropTableExpr, FlushTableExpr, GreptimeRequest, InsertRequests, PromRangeQuery,
QueryRequest, RequestHeader,
};
use arrow_flight::{FlightData, Ticket};
@@ -29,6 +29,9 @@ use common_telemetry::{logging, timer};
use futures_util::{TryFutureExt, TryStreamExt};
use prost::Message;
use snafu::{ensure, ResultExt};
use tokio::sync::mpsc::Sender;
use tokio::sync::{mpsc, OnceCell};
use tokio_stream::wrappers::ReceiverStream;
use crate::error::{
ConvertFlightDataSnafu, IllegalDatabaseResponseSnafu, IllegalFlightMessagesSnafu,
@@ -47,6 +50,7 @@ pub struct Database {
dbname: String,
client: Client,
streaming_client: OnceCell<Sender<GreptimeRequest>>,
ctx: FlightContext,
}
@@ -58,6 +62,7 @@ impl Database {
schema: schema.into(),
dbname: "".to_string(),
client,
streaming_client: OnceCell::new(),
ctx: FlightContext::default(),
}
}
@@ -75,6 +80,7 @@ impl Database {
schema: "".to_string(),
dbname: dbname.into(),
client,
streaming_client: OnceCell::new(),
ctx: FlightContext::default(),
}
}
@@ -109,9 +115,25 @@ impl Database {
});
}
pub async fn insert(&self, request: InsertRequest) -> Result<u32> {
pub async fn insert(&self, requests: InsertRequests) -> Result<u32> {
let _timer = timer!(metrics::METRIC_GRPC_INSERT);
self.handle(Request::Insert(request)).await
self.handle(Request::Inserts(requests)).await
}
pub async fn insert_to_stream(&self, requests: InsertRequests) -> Result<()> {
let streaming_client = self
.streaming_client
.get_or_try_init(|| self.client_stream())
.await?;
let request = self.to_rpc_request(Request::Inserts(requests));
streaming_client.send(request).await.map_err(|e| {
error::ClientStreamingSnafu {
err_msg: e.to_string(),
}
.build()
})
}
pub async fn delete(&self, request: DeleteRequest) -> Result<u32> {
@@ -121,15 +143,7 @@ impl Database {
async fn handle(&self, request: Request) -> Result<u32> {
let mut client = self.client.make_database_client()?.inner;
let request = GreptimeRequest {
header: Some(RequestHeader {
catalog: self.catalog.clone(),
schema: self.schema.clone(),
authorization: self.ctx.auth_header.clone(),
dbname: self.dbname.clone(),
}),
request: Some(request),
};
let request = self.to_rpc_request(request);
let response = client
.handle(request)
.await?
@@ -142,6 +156,27 @@ impl Database {
Ok(value)
}
#[inline]
fn to_rpc_request(&self, request: Request) -> GreptimeRequest {
GreptimeRequest {
header: Some(RequestHeader {
catalog: self.catalog.clone(),
schema: self.schema.clone(),
authorization: self.ctx.auth_header.clone(),
dbname: self.dbname.clone(),
}),
request: Some(request),
}
}
async fn client_stream(&self) -> Result<Sender<GreptimeRequest>> {
let mut client = self.client.make_database_client()?.inner;
let (sender, receiver) = mpsc::channel::<GreptimeRequest>(65536);
let receiver = ReceiverStream::new(receiver);
client.handle_requests(receiver).await?;
Ok(sender)
}
pub async fn sql(&self, sql: &str) -> Result<Output> {
let _timer = timer!(metrics::METRIC_GRPC_SQL);
self.do_get(Request::Query(QueryRequest {
@@ -212,15 +247,7 @@ impl Database {
async fn do_get(&self, request: Request) -> Result<Output> {
// FIXME(paomian): should be added some labels for metrics
let _timer = timer!(metrics::METRIC_GRPC_DO_GET);
let request = GreptimeRequest {
header: Some(RequestHeader {
catalog: self.catalog.clone(),
schema: self.schema.clone(),
authorization: self.ctx.auth_header.clone(),
dbname: self.dbname.clone(),
}),
request: Some(request),
};
let request = self.to_rpc_request(request);
let request = Ticket {
ticket: request.encode_to_vec().into(),
};

View File

@@ -67,6 +67,9 @@ pub enum Error {
#[snafu(display("Illegal Database response: {err_msg}"))]
IllegalDatabaseResponse { err_msg: String },
#[snafu(display("Failed to send request with streaming: {}", err_msg))]
ClientStreaming { err_msg: String, location: Location },
}
pub type Result<T> = std::result::Result<T, Error>;
@@ -77,7 +80,8 @@ impl ErrorExt for Error {
Error::IllegalFlightMessages { .. }
| Error::ColumnDataType { .. }
| Error::MissingField { .. }
| Error::IllegalDatabaseResponse { .. } => StatusCode::Internal,
| Error::IllegalDatabaseResponse { .. }
| Error::ClientStreaming { .. } => StatusCode::Internal,
Error::Server { code, .. } => *code,
Error::FlightGet { source, .. } => source.status_code(),

View File

@@ -13,6 +13,7 @@
// limitations under the License.
mod client;
pub mod client_manager;
mod database;
mod error;
pub mod load_balance;

View File

@@ -32,6 +32,7 @@ frontend = { path = "../frontend" }
futures.workspace = true
meta-client = { path = "../meta-client" }
meta-srv = { path = "../meta-srv" }
metrics.workspace = true
nu-ansi-term = "0.46"
partition = { path = "../partition" }
query = { path = "../query" }

View File

@@ -18,6 +18,10 @@ fn main() {
"cargo:rustc-env=GIT_COMMIT={}",
build_data::get_git_commit().unwrap_or_else(|_| DEFAULT_VALUE.to_string())
);
println!(
"cargo:rustc-env=GIT_COMMIT_SHORT={}",
build_data::get_git_commit_short().unwrap_or_else(|_| DEFAULT_VALUE.to_string())
);
println!(
"cargo:rustc-env=GIT_BRANCH={}",
build_data::get_git_branch().unwrap_or_else(|_| DEFAULT_VALUE.to_string())

View File

@@ -21,6 +21,7 @@ use cmd::error::Result;
use cmd::options::{Options, TopLevelOptions};
use cmd::{cli, datanode, frontend, metasrv, standalone};
use common_telemetry::logging::{error, info, TracingOptions};
use metrics::gauge;
#[derive(Parser)]
#[clap(name = "greptimedb", version = print_version())]
@@ -163,6 +164,22 @@ fn print_version() -> &'static str {
)
}
fn short_version() -> &'static str {
env!("CARGO_PKG_VERSION")
}
// {app_name}-{branch_name}-{commit_short}
// The branch name (tag) of a release build should already contain the short
// version so the full version doesn't concat the short version explicitly.
fn full_version() -> &'static str {
concat!(
"greptimedb-",
env!("GIT_BRANCH"),
"-",
env!("GIT_COMMIT_SHORT")
)
}
#[cfg(feature = "mem-prof")]
#[global_allocator]
static ALLOC: tikv_jemallocator::Jemalloc = tikv_jemallocator::Jemalloc;
@@ -185,6 +202,9 @@ async fn main() -> Result<()> {
common_telemetry::init_default_metrics_recorder();
let _guard = common_telemetry::init_global_logging(app_name, logging_opts, tracing_opts);
// Report app version as gauge.
gauge!("app_version", 1.0, "short_version" => short_version(), "version" => full_version());
let mut app = cmd.build(opts).await?;
tokio::select! {

View File

@@ -17,6 +17,7 @@ use std::sync::Arc;
use std::time::Instant;
use catalog::remote::CachedMetaKvBackend;
use client::client_manager::DatanodeClients;
use client::{Client, Database, DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
use common_error::prelude::ErrorExt;
use common_query::Output;
@@ -24,7 +25,6 @@ use common_recordbatch::RecordBatches;
use common_telemetry::logging;
use either::Either;
use frontend::catalog::FrontendCatalogManager;
use frontend::datanode::DatanodeClients;
use meta_client::client::MetaClientBuilder;
use partition::manager::PartitionRuleManager;
use partition::route::TableRoutes;
@@ -269,6 +269,8 @@ async fn create_query_engine(meta_addr: &str) -> Result<DatafusionQueryEngine> {
let state = Arc::new(QueryEngineState::new(
catalog_list,
false,
None,
None,
Default::default(),
));

View File

@@ -280,6 +280,7 @@ mod tests {
}
ObjectStoreConfig::S3 { .. } => unreachable!(),
ObjectStoreConfig::Oss { .. } => unreachable!(),
ObjectStoreConfig::Azblob { .. } => unreachable!(),
};
assert_eq!(

View File

@@ -16,6 +16,7 @@ use std::sync::Arc;
use clap::Parser;
use common_base::Plugins;
use common_telemetry::logging;
use frontend::frontend::FrontendOptions;
use frontend::instance::{FrontendInstance, Instance as FeInstance};
use frontend::service_config::{InfluxdbOptions, PromOptions};
@@ -202,6 +203,9 @@ impl StartCommand {
}
async fn build(self, opts: FrontendOptions) -> Result<Instance> {
logging::info!("Frontend start command: {:#?}", self);
logging::info!("Frontend options: {:#?}", opts);
let plugins = Arc::new(load_frontend_plugins(&self.user_provider)?);
let mut instance = FeInstance::try_new_distributed(&opts, plugins.clone())

View File

@@ -302,6 +302,7 @@ impl StartCommand {
async fn build(self, fe_opts: FrontendOptions, dn_opts: DatanodeOptions) -> Result<Instance> {
let plugins = Arc::new(load_frontend_plugins(&self.user_provider)?);
info!("Standalone start command: {:#?}", self);
info!(
"Standalone frontend options: {:#?}, datanode options: {:#?}",
fe_opts, dn_opts

View File

@@ -110,6 +110,7 @@ impl ArrowDecoder for arrow::csv::reader::Decoder {
}
}
#[allow(deprecated)]
impl ArrowDecoder for arrow::json::RawDecoder {
fn decode(&mut self, buf: &[u8]) -> result::Result<usize, ArrowError> {
self.decode(buf)

View File

@@ -17,6 +17,7 @@ use std::str::FromStr;
use std::sync::Arc;
use arrow::csv;
#[allow(deprecated)]
use arrow::csv::reader::infer_reader_schema as infer_csv_schema;
use arrow::record_batch::RecordBatch;
use arrow_schema::{Schema, SchemaRef};
@@ -113,8 +114,7 @@ pub struct CsvConfig {
impl CsvConfig {
fn builder(&self) -> csv::ReaderBuilder {
let mut builder = csv::ReaderBuilder::new()
.with_schema(self.file_schema.clone())
let mut builder = csv::ReaderBuilder::new(self.file_schema.clone())
.with_delimiter(self.delimiter)
.with_batch_size(self.batch_size)
.has_header(self.has_header);
@@ -160,6 +160,7 @@ impl FileOpener for CsvOpener {
}
}
#[allow(deprecated)]
#[async_trait]
impl FileFormat for CsvFormat {
async fn infer_schema(&self, store: &ObjectStore, path: &str) -> Result<Schema> {

View File

@@ -20,6 +20,7 @@ use std::sync::Arc;
use arrow::datatypes::SchemaRef;
use arrow::json::reader::{infer_json_schema_from_iterator, ValueIter};
use arrow::json::writer::LineDelimited;
#[allow(deprecated)]
use arrow::json::{self, RawReaderBuilder};
use arrow::record_batch::RecordBatch;
use arrow_schema::Schema;
@@ -129,6 +130,7 @@ impl JsonOpener {
}
}
#[allow(deprecated)]
impl FileOpener for JsonOpener {
fn open(&self, meta: FileMeta) -> DataFusionResult<FileOpenFuture> {
open_with_decoder(
@@ -159,8 +161,7 @@ pub async fn stream_to_json(
impl DfRecordBatchEncoder for json::Writer<SharedBuffer, LineDelimited> {
fn write(&mut self, batch: &RecordBatch) -> Result<()> {
self.write(batch.clone())
.context(error::WriteRecordBatchSnafu)
self.write(batch).context(error::WriteRecordBatchSnafu)
}
}

View File

@@ -17,3 +17,6 @@ common-time = { path = "../time" }
datatypes = { path = "../../datatypes" }
snafu = { version = "0.7", features = ["backtraces"] }
table = { path = "../../table" }
[dev-dependencies]
paste = "1.0"

View File

@@ -12,9 +12,12 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use api::v1::add_column::location::LocationType;
use api::v1::add_column::Location;
use api::v1::alter_expr::Kind;
use api::v1::{column_def, AlterExpr, CreateTableExpr, DropColumns, RenameTable};
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
use common_query::AddColumnLocation;
use datatypes::schema::{ColumnSchema, RawSchema};
use snafu::{ensure, OptionExt, ResultExt};
use table::metadata::TableId;
@@ -24,9 +27,12 @@ use table::requests::{
use crate::error::{
ColumnNotFoundSnafu, InvalidColumnDefSnafu, MissingFieldSnafu, MissingTimestampColumnSnafu,
Result, UnrecognizedTableOptionSnafu,
Result, UnknownLocationTypeSnafu, UnrecognizedTableOptionSnafu,
};
const LOCATION_TYPE_FIRST: i32 = LocationType::First as i32;
const LOCATION_TYPE_AFTER: i32 = LocationType::After as i32;
/// Convert an [`AlterExpr`] to an [`AlterTableRequest`]
pub fn alter_expr_to_request(expr: AlterExpr) -> Result<AlterTableRequest> {
let catalog_name = expr.catalog_name;
@@ -50,6 +56,7 @@ pub fn alter_expr_to_request(expr: AlterExpr) -> Result<AlterTableRequest> {
Ok(AddColumnRequest {
column_schema: schema,
is_key: ac.is_key,
location: parse_location(ac.location)?,
})
})
.collect::<Result<Vec<_>>>()?;
@@ -163,10 +170,10 @@ pub fn create_expr_to_request(
Some(expr.desc)
};
let region_ids = if expr.region_ids.is_empty() {
let region_numbers = if expr.region_numbers.is_empty() {
vec![0]
} else {
expr.region_ids
expr.region_numbers
};
let table_options =
@@ -178,7 +185,7 @@ pub fn create_expr_to_request(
table_name: expr.table_name,
desc,
schema,
region_numbers: region_ids,
region_numbers,
primary_key_indices,
create_if_not_exists: expr.create_if_not_exists,
table_options,
@@ -186,8 +193,26 @@ pub fn create_expr_to_request(
})
}
fn parse_location(location: Option<Location>) -> Result<Option<AddColumnLocation>> {
match location {
Some(Location {
location_type: LOCATION_TYPE_FIRST,
..
}) => Ok(Some(AddColumnLocation::First)),
Some(Location {
location_type: LOCATION_TYPE_AFTER,
after_cloumn_name,
}) => Ok(Some(AddColumnLocation::After {
column_name: after_cloumn_name,
})),
Some(Location { location_type, .. }) => UnknownLocationTypeSnafu { location_type }.fail(),
None => Ok(None),
}
}
#[cfg(test)]
mod tests {
use api::v1::add_column::location::LocationType;
use api::v1::{AddColumn, AddColumns, ColumnDataType, ColumnDef, DropColumn};
use datatypes::prelude::ConcreteDataType;
@@ -209,6 +234,7 @@ mod tests {
default_constraint: vec![],
}),
is_key: false,
location: None,
}],
})),
};
@@ -228,6 +254,80 @@ mod tests {
ConcreteDataType::float64_datatype(),
add_column.column_schema.data_type
);
assert_eq!(None, add_column.location);
}
#[test]
fn test_alter_expr_with_location_to_request() {
let expr = AlterExpr {
catalog_name: "".to_string(),
schema_name: "".to_string(),
table_name: "monitor".to_string(),
kind: Some(Kind::AddColumns(AddColumns {
add_columns: vec![
AddColumn {
column_def: Some(ColumnDef {
name: "mem_usage".to_string(),
datatype: ColumnDataType::Float64 as i32,
is_nullable: false,
default_constraint: vec![],
}),
is_key: false,
location: Some(Location {
location_type: LocationType::First.into(),
after_cloumn_name: "".to_string(),
}),
},
AddColumn {
column_def: Some(ColumnDef {
name: "cpu_usage".to_string(),
datatype: ColumnDataType::Float64 as i32,
is_nullable: false,
default_constraint: vec![],
}),
is_key: false,
location: Some(Location {
location_type: LocationType::After.into(),
after_cloumn_name: "ts".to_string(),
}),
},
],
})),
};
let alter_request = alter_expr_to_request(expr).unwrap();
assert_eq!(alter_request.catalog_name, "");
assert_eq!(alter_request.schema_name, "");
assert_eq!("monitor".to_string(), alter_request.table_name);
let mut add_columns = match alter_request.alter_kind {
AlterKind::AddColumns { columns } => columns,
_ => unreachable!(),
};
let add_column = add_columns.pop().unwrap();
assert!(!add_column.is_key);
assert_eq!("cpu_usage", add_column.column_schema.name);
assert_eq!(
ConcreteDataType::float64_datatype(),
add_column.column_schema.data_type
);
assert_eq!(
Some(AddColumnLocation::After {
column_name: "ts".to_string()
}),
add_column.location
);
let add_column = add_columns.pop().unwrap();
assert!(!add_column.is_key);
assert_eq!("mem_usage", add_column.column_schema.name);
assert_eq!(
ConcreteDataType::float64_datatype(),
add_column.column_schema.data_type
);
assert_eq!(Some(AddColumnLocation::First), add_column.location);
}
#[test]

View File

@@ -16,7 +16,6 @@ use std::collections::HashMap;
use api::helper::ColumnDataTypeWrapper;
use api::v1::{Column, DeleteRequest as GrpcDeleteRequest};
use datatypes::data_type::DataType;
use datatypes::prelude::ConcreteDataType;
use snafu::{ensure, ResultExt};
use table::requests::DeleteRequest;
@@ -41,14 +40,11 @@ pub fn to_table_delete_request(request: GrpcDeleteRequest) -> Result<DeleteReque
let datatype: ConcreteDataType = ColumnDataTypeWrapper::try_new(datatype)
.context(ColumnDataTypeSnafu)?
.into();
let vector_builder = &mut datatype.create_mutable_vector(row_count);
add_values_to_builder(vector_builder, values, row_count, null_mask)?;
let vector = add_values_to_builder(datatype, values, row_count, null_mask)?;
ensure!(
key_column_values
.insert(column_name.clone(), vector_builder.to_vector())
.insert(column_name.clone(), vector)
.is_none(),
IllegalDeleteRequestSnafu {
reason: format!("Duplicated column '{column_name}' in delete request.")

View File

@@ -83,6 +83,12 @@ pub enum Error {
#[snafu(display("The column name already exists, column: {}", column))]
ColumnAlreadyExists { column: String, location: Location },
#[snafu(display("Unknown location type: {}", location_type))]
UnknownLocationType {
location_type: i32,
location: Location,
},
}
pub type Result<T> = std::result::Result<T, Error>;
@@ -103,9 +109,9 @@ impl ErrorExt for Error {
Error::MissingField { .. } => StatusCode::InvalidArguments,
Error::InvalidColumnDef { source, .. } => source.status_code(),
Error::UnrecognizedTableOption { .. } => StatusCode::InvalidArguments,
Error::UnexpectedValuesLength { .. } | Error::ColumnAlreadyExists { .. } => {
StatusCode::InvalidArguments
}
Error::UnexpectedValuesLength { .. }
| Error::ColumnAlreadyExists { .. }
| Error::UnknownLocationType { .. } => StatusCode::InvalidArguments,
}
}

View File

@@ -13,6 +13,7 @@
// limitations under the License.
use std::collections::{HashMap, HashSet};
use std::sync::Arc;
use api::helper::ColumnDataTypeWrapper;
use api::v1::column::{SemanticType, Values};
@@ -25,10 +26,16 @@ use common_time::timestamp::Timestamp;
use common_time::{Date, DateTime};
use datatypes::data_type::{ConcreteDataType, DataType};
use datatypes::prelude::{ValueRef, VectorRef};
use datatypes::scalars::ScalarVector;
use datatypes::schema::SchemaRef;
use datatypes::types::TimestampType;
use datatypes::types::{Int16Type, Int8Type, TimestampType, UInt16Type, UInt8Type};
use datatypes::value::Value;
use datatypes::vectors::MutableVector;
use datatypes::vectors::{
BinaryVector, BooleanVector, DateTimeVector, DateVector, Float32Vector, Float64Vector,
Int32Vector, Int64Vector, PrimitiveVector, StringVector, TimestampMicrosecondVector,
TimestampMillisecondVector, TimestampNanosecondVector, TimestampSecondVector, UInt32Vector,
UInt64Vector,
};
use snafu::{ensure, OptionExt, ResultExt};
use table::metadata::TableId;
use table::requests::InsertRequest;
@@ -68,6 +75,7 @@ pub fn find_new_columns(schema: &SchemaRef, columns: &[Column]) -> Result<Option
columns_to_add.push(AddColumn {
column_def,
is_key: *semantic_type == TAG_SEMANTIC_TYPE,
location: None,
});
new_columns.insert(column_name.to_string());
}
@@ -257,7 +265,7 @@ pub fn build_create_expr_from_insertion(
create_if_not_exists: true,
table_options: Default::default(),
table_id: table_id.map(|id| api::v1::TableId { id }),
region_ids: vec![0], // TODO:(hl): region id should be allocated by frontend
region_numbers: vec![0], // TODO:(hl): region number should be allocated by frontend
engine: engine.to_string(),
};
@@ -286,15 +294,10 @@ pub fn to_table_insert_request(
let datatype: ConcreteDataType = ColumnDataTypeWrapper::try_new(datatype)
.context(ColumnDataTypeSnafu)?
.into();
let vector_builder = &mut datatype.create_mutable_vector(row_count);
add_values_to_builder(vector_builder, values, row_count, null_mask)?;
let vector = add_values_to_builder(datatype, values, row_count, null_mask)?;
ensure!(
columns_values
.insert(column_name.clone(), vector_builder.to_vector())
.is_none(),
columns_values.insert(column_name.clone(), vector).is_none(),
ColumnAlreadyExistsSnafu {
column: column_name
}
@@ -311,28 +314,16 @@ pub fn to_table_insert_request(
}
pub(crate) fn add_values_to_builder(
builder: &mut Box<dyn MutableVector>,
data_type: ConcreteDataType,
values: Values,
row_count: usize,
null_mask: Vec<u8>,
) -> Result<()> {
let data_type = builder.data_type();
let values = convert_values(&data_type, values);
) -> Result<VectorRef> {
if null_mask.is_empty() {
ensure!(
values.len() == row_count,
UnexpectedValuesLengthSnafu {
reason: "If null_mask is empty, the length of values must be equal to row_count."
}
);
values.iter().try_for_each(|value| {
builder
.try_push_value_ref(value.as_value_ref())
.context(CreateVectorSnafu)
})?;
Ok(values_to_vector(&data_type, values))
} else {
let builder = &mut data_type.create_mutable_vector(row_count);
let values = convert_values(&data_type, values);
let null_mask = BitVec::from_vec(null_mask);
ensure!(
null_mask.count_ones() + values.len() == row_count,
@@ -353,8 +344,53 @@ pub(crate) fn add_values_to_builder(
}
}
}
Ok(builder.to_vector())
}
}
fn values_to_vector(data_type: &ConcreteDataType, values: Values) -> VectorRef {
match data_type {
ConcreteDataType::Boolean(_) => Arc::new(BooleanVector::from(values.bool_values)),
ConcreteDataType::Int8(_) => Arc::new(PrimitiveVector::<Int8Type>::from_iter_values(
values.i8_values.into_iter().map(|x| x as i8),
)),
ConcreteDataType::Int16(_) => Arc::new(PrimitiveVector::<Int16Type>::from_iter_values(
values.i16_values.into_iter().map(|x| x as i16),
)),
ConcreteDataType::Int32(_) => Arc::new(Int32Vector::from_vec(values.i32_values)),
ConcreteDataType::Int64(_) => Arc::new(Int64Vector::from_vec(values.i64_values)),
ConcreteDataType::UInt8(_) => Arc::new(PrimitiveVector::<UInt8Type>::from_iter_values(
values.u8_values.into_iter().map(|x| x as u8),
)),
ConcreteDataType::UInt16(_) => Arc::new(PrimitiveVector::<UInt16Type>::from_iter_values(
values.u16_values.into_iter().map(|x| x as u16),
)),
ConcreteDataType::UInt32(_) => Arc::new(UInt32Vector::from_vec(values.u32_values)),
ConcreteDataType::UInt64(_) => Arc::new(UInt64Vector::from_vec(values.u64_values)),
ConcreteDataType::Float32(_) => Arc::new(Float32Vector::from_vec(values.f32_values)),
ConcreteDataType::Float64(_) => Arc::new(Float64Vector::from_vec(values.f64_values)),
ConcreteDataType::Binary(_) => Arc::new(BinaryVector::from(values.binary_values)),
ConcreteDataType::String(_) => Arc::new(StringVector::from_vec(values.string_values)),
ConcreteDataType::Date(_) => Arc::new(DateVector::from_vec(values.date_values)),
ConcreteDataType::DateTime(_) => Arc::new(DateTimeVector::from_vec(values.datetime_values)),
ConcreteDataType::Timestamp(unit) => match unit {
TimestampType::Second(_) => {
Arc::new(TimestampSecondVector::from_vec(values.ts_second_values))
}
TimestampType::Millisecond(_) => Arc::new(TimestampMillisecondVector::from_vec(
values.ts_millisecond_values,
)),
TimestampType::Microsecond(_) => Arc::new(TimestampMicrosecondVector::from_vec(
values.ts_microsecond_values,
)),
TimestampType::Nanosecond(_) => Arc::new(TimestampNanosecondVector::from_vec(
values.ts_nanosecond_values,
)),
},
ConcreteDataType::Null(_) | ConcreteDataType::List(_) | ConcreteDataType::Dictionary(_) => {
unreachable!()
}
}
Ok(())
}
fn convert_values(data_type: &ConcreteDataType, values: Values) -> Vec<Value> {
@@ -380,22 +416,34 @@ fn convert_values(data_type: &ConcreteDataType, values: Values) -> Vec<Value> {
.into_iter()
.map(|val| val.into())
.collect(),
ConcreteDataType::Int8(_) => values.i8_values.into_iter().map(|val| val.into()).collect(),
ConcreteDataType::Int8(_) => values
.i8_values
.into_iter()
// Safety: Since i32 only stores i8 data here, so i32 as i8 is safe.
.map(|val| (val as i8).into())
.collect(),
ConcreteDataType::Int16(_) => values
.i16_values
.into_iter()
.map(|val| val.into())
// Safety: Since i32 only stores i16 data here, so i32 as i16 is safe.
.map(|val| (val as i16).into())
.collect(),
ConcreteDataType::Int32(_) => values
.i32_values
.into_iter()
.map(|val| val.into())
.collect(),
ConcreteDataType::UInt8(_) => values.u8_values.into_iter().map(|val| val.into()).collect(),
ConcreteDataType::UInt8(_) => values
.u8_values
.into_iter()
// Safety: Since i32 only stores u8 data here, so i32 as u8 is safe.
.map(|val| (val as u8).into())
.collect(),
ConcreteDataType::UInt16(_) => values
.u16_values
.into_iter()
.map(|val| val.into())
// Safety: Since i32 only stores u16 data here, so i32 as u16 is safe.
.map(|val| (val as u16).into())
.collect(),
ConcreteDataType::UInt32(_) => values
.u32_values
@@ -418,12 +466,12 @@ fn convert_values(data_type: &ConcreteDataType, values: Values) -> Vec<Value> {
.map(|val| val.into())
.collect(),
ConcreteDataType::DateTime(_) => values
.i64_values
.datetime_values
.into_iter()
.map(|v| Value::DateTime(v.into()))
.collect(),
ConcreteDataType::Date(_) => values
.i32_values
.date_values
.into_iter()
.map(|v| Value::Date(v.into()))
.collect(),
@@ -472,6 +520,7 @@ mod tests {
use datatypes::schema::{ColumnSchema, SchemaBuilder};
use datatypes::types::{TimestampMillisecondType, TimestampSecondType, TimestampType};
use datatypes::value::Value;
use paste::paste;
use snafu::ResultExt;
use super::*;
@@ -660,26 +709,150 @@ mod tests {
assert_eq!(Value::Timestamp(Timestamp::new_millisecond(101)), ts.get(1));
}
#[test]
fn test_convert_values() {
let data_type = ConcreteDataType::float64_datatype();
let values = Values {
f64_values: vec![0.1, 0.2, 0.3],
..Default::default()
macro_rules! test_convert_values {
($grpc_data_type: ident, $values: expr, $concrete_data_type: ident, $expected_ret: expr) => {
paste! {
#[test]
fn [<test_convert_ $grpc_data_type _values>]() {
let values = Values {
[<$grpc_data_type _values>]: $values,
..Default::default()
};
let data_type = ConcreteDataType::[<$concrete_data_type _datatype>]();
let result = convert_values(&data_type, values);
assert_eq!(
$expected_ret,
result
);
}
}
};
let result = convert_values(&data_type, values);
assert_eq!(
vec![
Value::Float64(0.1.into()),
Value::Float64(0.2.into()),
Value::Float64(0.3.into())
],
result
);
}
test_convert_values!(
i8,
vec![1_i32, 2, 3],
int8,
vec![Value::Int8(1), Value::Int8(2), Value::Int8(3)]
);
test_convert_values!(
u8,
vec![1_u32, 2, 3],
uint8,
vec![Value::UInt8(1), Value::UInt8(2), Value::UInt8(3)]
);
test_convert_values!(
i16,
vec![1_i32, 2, 3],
int16,
vec![Value::Int16(1), Value::Int16(2), Value::Int16(3)]
);
test_convert_values!(
u16,
vec![1_u32, 2, 3],
uint16,
vec![Value::UInt16(1), Value::UInt16(2), Value::UInt16(3)]
);
test_convert_values!(
i32,
vec![1, 2, 3],
int32,
vec![Value::Int32(1), Value::Int32(2), Value::Int32(3)]
);
test_convert_values!(
u32,
vec![1, 2, 3],
uint32,
vec![Value::UInt32(1), Value::UInt32(2), Value::UInt32(3)]
);
test_convert_values!(
i64,
vec![1, 2, 3],
int64,
vec![Value::Int64(1), Value::Int64(2), Value::Int64(3)]
);
test_convert_values!(
u64,
vec![1, 2, 3],
uint64,
vec![Value::UInt64(1), Value::UInt64(2), Value::UInt64(3)]
);
test_convert_values!(
f32,
vec![1.0, 2.0, 3.0],
float32,
vec![
Value::Float32(1.0.into()),
Value::Float32(2.0.into()),
Value::Float32(3.0.into())
]
);
test_convert_values!(
f64,
vec![1.0, 2.0, 3.0],
float64,
vec![
Value::Float64(1.0.into()),
Value::Float64(2.0.into()),
Value::Float64(3.0.into())
]
);
test_convert_values!(
string,
vec!["1".to_string(), "2".to_string(), "3".to_string()],
string,
vec![
Value::String("1".into()),
Value::String("2".into()),
Value::String("3".into())
]
);
test_convert_values!(
binary,
vec!["1".into(), "2".into(), "3".into()],
binary,
vec![
Value::Binary(b"1".to_vec().into()),
Value::Binary(b"2".to_vec().into()),
Value::Binary(b"3".to_vec().into())
]
);
test_convert_values!(
date,
vec![1, 2, 3],
date,
vec![
Value::Date(1.into()),
Value::Date(2.into()),
Value::Date(3.into())
]
);
test_convert_values!(
datetime,
vec![1.into(), 2.into(), 3.into()],
datetime,
vec![
Value::DateTime(1.into()),
Value::DateTime(2.into()),
Value::DateTime(3.into())
]
);
#[test]
fn test_convert_timestamp_values() {
// second

View File

@@ -13,6 +13,7 @@ common-base = { path = "../base" }
common-error = { path = "../error" }
common-function-macro = { path = "../function-macro" }
common-query = { path = "../query" }
common-meta = { path = "../meta" }
common-recordbatch = { path = "../recordbatch" }
common-runtime = { path = "../runtime" }
common-telemetry = { path = "../telemetry" }

View File

@@ -28,6 +28,7 @@ use tower::make::MakeConnection;
use crate::error::{CreateChannelSnafu, InvalidConfigFilePathSnafu, InvalidTlsConfigSnafu, Result};
const RECYCLE_CHANNEL_INTERVAL_SECS: u64 = 60;
const DEFAULT_REQUEST_TIMEOUT_SECS: u64 = 2;
#[derive(Clone, Debug)]
pub struct ChannelManager {
@@ -236,7 +237,7 @@ pub struct ChannelConfig {
impl Default for ChannelConfig {
fn default() -> Self {
Self {
timeout: Some(Duration::from_secs(2)),
timeout: Some(Duration::from_secs(DEFAULT_REQUEST_TIMEOUT_SECS)),
connect_timeout: Some(Duration::from_secs(4)),
concurrency_limit: None,
rate_limit: None,
@@ -497,7 +498,7 @@ mod tests {
let default_cfg = ChannelConfig::new();
assert_eq!(
ChannelConfig {
timeout: Some(Duration::from_secs(2)),
timeout: Some(Duration::from_secs(DEFAULT_REQUEST_TIMEOUT_SECS)),
connect_timeout: Some(Duration::from_secs(4)),
concurrency_limit: None,
rate_limit: None,

View File

@@ -12,8 +12,8 @@ common-runtime = { path = "../runtime" }
common-telemetry = { path = "../telemetry" }
common-time = { path = "../time" }
serde.workspace = true
snafu.workspace = true
serde_json.workspace = true
snafu.workspace = true
store-api = { path = "../../store-api" }
table = { path = "../../table" }
tokio.workspace = true

View File

@@ -22,11 +22,7 @@ use crate::{ClusterId, DatanodeId};
pub struct RegionIdent {
pub cluster_id: ClusterId,
pub datanode_id: DatanodeId,
pub catalog: String,
pub schema: String,
pub table: String,
pub table_id: u32,
pub engine: String,
pub table_ident: TableIdent,
pub region_number: u32,
}
@@ -37,20 +33,24 @@ impl Display for RegionIdent {
"RegionIdent(datanode_id='{}.{}', table_id='{}', table_name='{}.{}.{}', table_engine='{}', region_no='{}')",
self.cluster_id,
self.datanode_id,
self.table_id,
self.catalog,
self.schema,
self.table,
self.engine,
self.table_ident.table_id,
self.table_ident.catalog,
self.table_ident.schema,
self.table_ident.table,
self.table_ident.engine,
self.region_number
)
}
}
impl From<RegionIdent> for TableIdent {
fn from(region_ident: RegionIdent) -> Self {
region_ident.table_ident
}
}
#[derive(Eq, Hash, PartialEq, Clone, Debug, Serialize, Deserialize)]
pub struct TableIdent {
pub cluster_id: ClusterId,
pub datanode_id: DatanodeId,
pub catalog: String,
pub schema: String,
pub table: String,
@@ -62,14 +62,8 @@ impl Display for TableIdent {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
write!(
f,
"TableIdent(datanode_id='{}.{}', table_id='{}', table_name='{}.{}.{}', table_engine='{}')",
self.cluster_id,
self.datanode_id,
self.table_id,
self.catalog,
self.schema,
self.table,
self.engine,
"TableIdent(table_id='{}', table_name='{}.{}.{}', table_engine='{}')",
self.table_id, self.catalog, self.schema, self.table, self.engine,
)
}
}
@@ -86,7 +80,7 @@ impl Display for SimpleReply {
}
}
#[derive(Debug, Serialize, Deserialize)]
#[derive(Debug, Clone, Serialize, Deserialize)]
#[serde(tag = "type", rename_all = "snake_case")]
pub enum Instruction {
OpenRegion(RegionIdent),
@@ -133,36 +127,40 @@ mod tests {
let open_region = Instruction::OpenRegion(RegionIdent {
cluster_id: 1,
datanode_id: 2,
catalog: "foo".to_string(),
schema: "bar".to_string(),
table: "hi".to_string(),
table_id: 1024,
engine: "mito".to_string(),
table_ident: TableIdent {
catalog: "foo".to_string(),
schema: "bar".to_string(),
table: "hi".to_string(),
table_id: 1024,
engine: "mito".to_string(),
},
region_number: 1,
});
let serialized = serde_json::to_string(&open_region).unwrap();
assert_eq!(
r#"{"type":"open_region","cluster_id":1,"datanode_id":2,"catalog":"foo","schema":"bar","table":"hi","table_id":1024,"engine":"mito","region_number":1}"#,
r#"{"type":"open_region","cluster_id":1,"datanode_id":2,"table_ident":{"catalog":"foo","schema":"bar","table":"hi","table_id":1024,"engine":"mito"},"region_number":1}"#,
serialized
);
let close_region = Instruction::CloseRegion(RegionIdent {
cluster_id: 1,
datanode_id: 2,
catalog: "foo".to_string(),
schema: "bar".to_string(),
table: "hi".to_string(),
table_id: 1024,
engine: "mito".to_string(),
table_ident: TableIdent {
catalog: "foo".to_string(),
schema: "bar".to_string(),
table: "hi".to_string(),
table_id: 1024,
engine: "mito".to_string(),
},
region_number: 1,
});
let serialized = serde_json::to_string(&close_region).unwrap();
assert_eq!(
r#"{"type":"close_region","cluster_id":1,"datanode_id":2,"catalog":"foo","schema":"bar","table":"hi","table_id":1024,"engine":"mito","region_number":1}"#,
r#"{"type":"close_region","cluster_id":1,"datanode_id":2,"table_ident":{"catalog":"foo","schema":"bar","table":"hi","table_id":1024,"engine":"mito"},"region_number":1}"#,
serialized
);
}

View File

@@ -0,0 +1,35 @@
// 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.
mod table_route;
pub use crate::key::table_route::{TableRouteKey, TABLE_ROUTE_PREFIX};
pub const REMOVED_PREFIX: &str = "__removed";
pub fn to_removed_key(key: &str) -> String {
format!("{REMOVED_PREFIX}-{key}")
}
#[cfg(test)]
mod tests {
use crate::key::to_removed_key;
#[test]
fn test_to_removed_key() {
let key = "test_key";
let removed = "__removed-test_key";
assert_eq!(removed, to_removed_key(key));
}
}

View File

@@ -0,0 +1,97 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use api::v1::meta::TableName;
use crate::key::to_removed_key;
pub const TABLE_ROUTE_PREFIX: &str = "__meta_table_route";
pub struct TableRouteKey<'a> {
pub table_id: u64,
pub catalog_name: &'a str,
pub schema_name: &'a str,
pub table_name: &'a str,
}
impl<'a> TableRouteKey<'a> {
pub fn with_table_name(table_id: u64, t: &'a TableName) -> Self {
Self {
table_id,
catalog_name: &t.catalog_name,
schema_name: &t.schema_name,
table_name: &t.table_name,
}
}
pub fn prefix(&self) -> String {
format!(
"{}-{}-{}-{}",
TABLE_ROUTE_PREFIX, self.catalog_name, self.schema_name, self.table_name
)
}
pub fn key(&self) -> String {
format!("{}-{}", self.prefix(), self.table_id)
}
pub fn removed_key(&self) -> String {
to_removed_key(&self.key())
}
}
#[cfg(test)]
mod tests {
use api::v1::meta::TableName;
use super::TableRouteKey;
#[test]
fn test_table_route_key() {
let key = TableRouteKey {
table_id: 123,
catalog_name: "greptime",
schema_name: "public",
table_name: "demo",
};
let prefix = key.prefix();
assert_eq!("__meta_table_route-greptime-public-demo", prefix);
let key_string = key.key();
assert_eq!("__meta_table_route-greptime-public-demo-123", key_string);
let removed = key.removed_key();
assert_eq!(
"__removed-__meta_table_route-greptime-public-demo-123",
removed
);
}
#[test]
fn test_with_table_name() {
let table_name = TableName {
catalog_name: "greptime".to_string(),
schema_name: "public".to_string(),
table_name: "demo".to_string(),
};
let key = TableRouteKey::with_table_name(123, &table_name);
assert_eq!(123, key.table_id);
assert_eq!("greptime", key.catalog_name);
assert_eq!("public", key.schema_name);
assert_eq!("demo", key.table_name);
}
}

View File

@@ -15,6 +15,7 @@
pub mod error;
pub mod heartbeat;
pub mod instruction;
pub mod key;
pub mod peer;
pub mod rpc;
pub mod table_name;

View File

@@ -12,11 +12,14 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::fmt::{Display, Formatter};
use api::v1::meta::Peer as PbPeer;
use serde::{Deserialize, Serialize};
#[derive(Debug, Clone, Hash, Eq, PartialEq, Deserialize, Serialize)]
pub struct Peer {
/// Node identifier. Unique in a cluster.
pub id: u64,
pub addr: String,
}
@@ -47,3 +50,9 @@ impl Peer {
}
}
}
impl Display for Peer {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
write!(f, "peer-{}({})", self.id, self.addr)
}
}

View File

@@ -22,6 +22,7 @@ use api::v1::meta::{
};
use serde::{Deserialize, Serialize, Serializer};
use snafu::{OptionExt, ResultExt};
use store_api::storage::{RegionId, RegionNumber};
use table::metadata::RawTableInfo;
use crate::error::{self, Result};
@@ -140,9 +141,23 @@ impl TryFrom<PbRouteResponse> for RouteResponse {
pub struct TableRoute {
pub table: Table,
pub region_routes: Vec<RegionRoute>,
region_leaders: HashMap<RegionNumber, Option<Peer>>,
}
impl TableRoute {
pub fn new(table: Table, region_routes: Vec<RegionRoute>) -> Self {
let region_leaders = region_routes
.iter()
.map(|x| (x.region.id as RegionNumber, x.leader_peer.clone()))
.collect::<HashMap<_, _>>();
Self {
table,
region_routes,
region_leaders,
}
}
pub fn try_from_raw(peers: &[PbPeer], table_route: PbTableRoute) -> Result<Self> {
let table = table_route
.table
@@ -178,10 +193,7 @@ impl TableRoute {
});
}
Ok(Self {
table,
region_routes,
})
Ok(Self::new(table, region_routes))
}
pub fn try_into_raw(self) -> Result<(Vec<PbPeer>, PbTableRoute)> {
@@ -266,6 +278,12 @@ impl TableRoute {
})
.collect()
}
pub fn find_region_leader(&self, region_number: RegionNumber) -> Option<&Peer> {
self.region_leaders
.get(&region_number)
.and_then(|x| x.as_ref())
}
}
#[derive(Debug, Clone, Deserialize, Serialize, PartialEq)]
@@ -313,7 +331,7 @@ pub struct RegionRoute {
#[derive(Debug, Clone, Default, Deserialize, Serialize, PartialEq)]
pub struct Region {
pub id: u64,
pub id: RegionId,
pub name: String,
pub partition: Option<Partition>,
pub attrs: HashMap<String, String>,
@@ -669,6 +687,10 @@ mod tests {
follower_peers: vec![Peer::new(2, "a2"), Peer::new(3, "a3")],
},
],
region_leaders: HashMap::from([
(2, Some(Peer::new(1, "a1"))),
(1, Some(Peer::new(2, "a2"))),
]),
};
let from_raw = TableRoute::try_from_raw(&raw_peers, raw_table_route.clone()).unwrap();

View File

@@ -127,6 +127,13 @@ pub enum Error {
source: common_runtime::error::Error,
location: Location,
},
#[snafu(display("Subprocedure {} failed, source: {}", subprocedure_id, source))]
SubprocedureFailed {
subprocedure_id: ProcedureId,
source: Arc<Error>,
location: Location,
},
}
pub type Result<T> = std::result::Result<T, Error>;
@@ -134,7 +141,7 @@ pub type Result<T> = std::result::Result<T, Error>;
impl ErrorExt for Error {
fn status_code(&self) -> StatusCode {
match self {
Error::External { source }
Error::External { source, .. }
| Error::PutState { source, .. }
| Error::DeleteStates { source, .. }
| Error::ListState { source, .. } => source.status_code(),
@@ -152,6 +159,8 @@ impl ErrorExt for Error {
Error::ProcedureExec { source, .. } => source.status_code(),
Error::StartRemoveOutdatedMetaTask { source, .. }
| Error::StopRemoveOutdatedMetaTask { source, .. } => source.status_code(),
Error::SubprocedureFailed { source, .. } => source.status_code(),
}
}

View File

@@ -5,6 +5,7 @@ edition.workspace = true
license.workspace = true
[dependencies]
api = { path = "../../api" }
async-trait.workspace = true
common-error = { path = "../error" }
common-recordbatch = { path = "../recordbatch" }
@@ -13,6 +14,7 @@ datafusion.workspace = true
datafusion-common.workspace = true
datafusion-expr.workspace = true
datatypes = { path = "../../datatypes" }
serde.workspace = true
snafu.workspace = true
statrs = "0.16"

View File

@@ -14,7 +14,10 @@
use std::fmt::{Debug, Formatter};
use api::greptime_proto::v1::add_column::location::LocationType;
use api::greptime_proto::v1::add_column::Location;
use common_recordbatch::{RecordBatches, SendableRecordBatchStream};
use serde::{Deserialize, Serialize};
pub mod columnar_value;
pub mod error;
@@ -44,3 +47,24 @@ impl Debug for Output {
}
pub use datafusion::physical_plan::ExecutionPlan as DfPhysicalPlan;
#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)]
pub enum AddColumnLocation {
First,
After { column_name: String },
}
impl From<&AddColumnLocation> for Location {
fn from(value: &AddColumnLocation) -> Self {
match value {
AddColumnLocation::First => Location {
location_type: LocationType::First.into(),
after_cloumn_name: "".to_string(),
},
AddColumnLocation::After { column_name } => Location {
location_type: LocationType::After.into(),
after_cloumn_name: column_name.to_string(),
},
}
}
}

View File

@@ -32,12 +32,12 @@ use crate::DfPhysicalPlan;
pub type PhysicalPlanRef = Arc<dyn PhysicalPlan>;
/// `PhysicalPlan` represent nodes in the Physical Plan.
/// [`PhysicalPlan`] represent nodes in the Physical Plan.
///
/// Each `PhysicalPlan` is Partition-aware and is responsible for
/// creating the actual `async` [`SendableRecordBatchStream`]s
/// of [`RecordBatch`] that incrementally compute the operator's
/// output from its input partition.
/// Each [`PhysicalPlan`] is partition-aware and is responsible for
/// creating the actual "async" [`SendableRecordBatchStream`]s
/// of [`RecordBatch`](common_recordbatch::RecordBatch) that incrementally
/// compute the operator's output from its input partition.
pub trait PhysicalPlan: Debug + Send + Sync {
/// Returns the physical plan as [`Any`](std::any::Any) so that it can be
/// downcast to a specific implementation.

View File

@@ -43,9 +43,9 @@ pub trait RecordBatchStream: Stream<Item = Result<RecordBatch>> {
pub type SendableRecordBatchStream = Pin<Box<dyn RecordBatchStream + Send>>;
#[derive(Debug, Clone, Copy)]
#[derive(Debug, Clone)]
pub struct OrderOption {
pub index: usize,
pub name: String,
pub options: SortOptions,
}
@@ -167,6 +167,15 @@ impl RecordBatches {
}
}
impl IntoIterator for RecordBatches {
type Item = RecordBatch;
type IntoIter = std::vec::IntoIter<Self::Item>;
fn into_iter(self) -> Self::IntoIter {
self.batches.into_iter()
}
}
pub struct SimpleRecordBatchStream {
inner: RecordBatches,
index: usize,
@@ -196,12 +205,17 @@ impl Stream for SimpleRecordBatchStream {
pub struct RecordBatchStreamAdaptor {
pub schema: SchemaRef,
pub stream: Pin<Box<dyn Stream<Item = Result<RecordBatch>> + Send>>,
pub output_ordering: Option<Vec<OrderOption>>,
}
impl RecordBatchStream for RecordBatchStreamAdaptor {
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
fn output_ordering(&self) -> Option<&[OrderOption]> {
self.output_ordering.as_deref()
}
}
impl Stream for RecordBatchStreamAdaptor {

View File

@@ -21,11 +21,10 @@ prost.workspace = true
session = { path = "../../session" }
snafu.workspace = true
table = { path = "../../table" }
query = { path = "../../query" }
[dependencies.substrait_proto]
package = "substrait"
version = "0.7"
version = "0.10"
[dev-dependencies]
datatypes = { path = "../../datatypes" }

View File

@@ -1,77 +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;
use datafusion::common::DFSchemaRef;
use substrait_proto::proto::extensions::simple_extension_declaration::{
ExtensionFunction, MappingType,
};
use substrait_proto::proto::extensions::SimpleExtensionDeclaration;
#[derive(Default)]
pub struct ConvertorContext {
scalar_fn_names: HashMap<String, u32>,
scalar_fn_map: HashMap<u32, String>,
df_schema: Option<DFSchemaRef>,
}
impl ConvertorContext {
pub fn register_scalar_fn<S: AsRef<str>>(&mut self, name: S) -> u32 {
if let Some(anchor) = self.scalar_fn_names.get(name.as_ref()) {
return *anchor;
}
let next_anchor = self.scalar_fn_map.len() as _;
self.scalar_fn_map
.insert(next_anchor, name.as_ref().to_string());
self.scalar_fn_names
.insert(name.as_ref().to_string(), next_anchor);
next_anchor
}
pub fn register_scalar_with_anchor<S: AsRef<str>>(&mut self, name: S, anchor: u32) {
self.scalar_fn_map.insert(anchor, name.as_ref().to_string());
self.scalar_fn_names
.insert(name.as_ref().to_string(), anchor);
}
pub fn find_scalar_fn(&self, anchor: u32) -> Option<&str> {
self.scalar_fn_map.get(&anchor).map(|s| s.as_str())
}
pub fn generate_function_extension(&self) -> Vec<SimpleExtensionDeclaration> {
let mut result = Vec::with_capacity(self.scalar_fn_map.len());
for (anchor, name) in &self.scalar_fn_map {
let declaration = SimpleExtensionDeclaration {
mapping_type: Some(MappingType::ExtensionFunction(ExtensionFunction {
extension_uri_reference: 0,
function_anchor: *anchor,
name: name.clone(),
})),
};
result.push(declaration);
}
result
}
pub(crate) fn set_df_schema(&mut self, schema: DFSchemaRef) {
debug_assert!(self.df_schema.is_none());
self.df_schema.get_or_insert(schema);
}
pub(crate) fn df_schema(&self) -> Option<&DFSchemaRef> {
self.df_schema.as_ref()
}
}

View File

@@ -1,799 +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::VecDeque;
use std::str::FromStr;
use datafusion::common::Column;
use datafusion_expr::expr::Sort;
use datafusion_expr::{expr_fn, lit, Between, BinaryExpr, BuiltinScalarFunction, Expr, Operator};
use datatypes::schema::Schema;
use snafu::{ensure, OptionExt};
use substrait_proto::proto::expression::field_reference::ReferenceType as FieldReferenceType;
use substrait_proto::proto::expression::reference_segment::{
ReferenceType as SegReferenceType, StructField,
};
use substrait_proto::proto::expression::{
FieldReference, Literal, ReferenceSegment, RexType, ScalarFunction,
};
use substrait_proto::proto::function_argument::ArgType;
use substrait_proto::proto::Expression;
use crate::context::ConvertorContext;
use crate::error::{
EmptyExprSnafu, InvalidParametersSnafu, MissingFieldSnafu, Result, UnsupportedExprSnafu,
};
use crate::types::{literal_type_to_scalar_value, scalar_value_as_literal_type};
/// Convert substrait's `Expression` to DataFusion's `Expr`.
pub(crate) fn to_df_expr(
ctx: &ConvertorContext,
expression: Expression,
schema: &Schema,
) -> Result<Expr> {
let expr_rex_type = expression.rex_type.context(EmptyExprSnafu)?;
match expr_rex_type {
RexType::Literal(l) => {
let t = l.literal_type.context(MissingFieldSnafu {
field: "LiteralType",
plan: "Literal",
})?;
let v = literal_type_to_scalar_value(t)?;
Ok(lit(v))
}
RexType::Selection(selection) => convert_selection_rex(*selection, schema),
RexType::ScalarFunction(scalar_fn) => convert_scalar_function(ctx, scalar_fn, schema),
RexType::WindowFunction(_)
| RexType::IfThen(_)
| RexType::SwitchExpression(_)
| RexType::SingularOrList(_)
| RexType::MultiOrList(_)
| RexType::Cast(_)
| RexType::Subquery(_)
| RexType::Nested(_)
| RexType::Enum(_) => UnsupportedExprSnafu {
name: format!("substrait expression {expr_rex_type:?}"),
}
.fail()?,
}
}
/// Convert Substrait's `FieldReference` - `DirectReference` - `StructField` to Datafusion's
/// `Column` expr.
pub fn convert_selection_rex(selection: FieldReference, schema: &Schema) -> Result<Expr> {
if let Some(FieldReferenceType::DirectReference(direct_ref)) = selection.reference_type
&& let Some(SegReferenceType::StructField(field)) = direct_ref.reference_type {
let column_name = schema.column_name_by_index(field.field as _).to_string();
Ok(Expr::Column(Column {
relation: None,
name: column_name,
}))
} else {
InvalidParametersSnafu {
reason: "Only support direct struct reference in Selection Rex",
}
.fail()
}
}
pub fn convert_scalar_function(
ctx: &ConvertorContext,
scalar_fn: ScalarFunction,
schema: &Schema,
) -> Result<Expr> {
// convert argument
let mut inputs = VecDeque::with_capacity(scalar_fn.arguments.len());
for arg in scalar_fn.arguments {
if let Some(ArgType::Value(sub_expr)) = arg.arg_type {
inputs.push_back(to_df_expr(ctx, sub_expr, schema)?);
} else {
InvalidParametersSnafu {
reason: "Only value expression arg is supported to be function argument",
}
.fail()?;
}
}
// convert this scalar function
// map function name
let anchor = scalar_fn.function_reference;
let fn_name = ctx
.find_scalar_fn(anchor)
.with_context(|| InvalidParametersSnafu {
reason: format!("Unregistered scalar function reference: {anchor}"),
})?;
// convenient util
let ensure_arg_len = |expected: usize| -> Result<()> {
ensure!(
inputs.len() == expected,
InvalidParametersSnafu {
reason: format!(
"Invalid number of scalar function {}, expected {} but found {}",
fn_name,
expected,
inputs.len()
)
}
);
Ok(())
};
// construct DataFusion expr
let expr = match fn_name {
// begin binary exprs, with the same order of DF `Operator`'s definition.
"eq" | "equal" => {
ensure_arg_len(2)?;
inputs.pop_front().unwrap().eq(inputs.pop_front().unwrap())
}
"not_eq" | "not_equal" => {
ensure_arg_len(2)?;
inputs
.pop_front()
.unwrap()
.not_eq(inputs.pop_front().unwrap())
}
"lt" => {
ensure_arg_len(2)?;
inputs.pop_front().unwrap().lt(inputs.pop_front().unwrap())
}
"lt_eq" | "lte" => {
ensure_arg_len(2)?;
inputs
.pop_front()
.unwrap()
.lt_eq(inputs.pop_front().unwrap())
}
"gt" => {
ensure_arg_len(2)?;
inputs.pop_front().unwrap().gt(inputs.pop_front().unwrap())
}
"gt_eq" | "gte" => {
ensure_arg_len(2)?;
inputs
.pop_front()
.unwrap()
.gt_eq(inputs.pop_front().unwrap())
}
"plus" => {
ensure_arg_len(2)?;
expr_fn::binary_expr(
inputs.pop_front().unwrap(),
Operator::Plus,
inputs.pop_front().unwrap(),
)
}
"minus" => {
ensure_arg_len(2)?;
expr_fn::binary_expr(
inputs.pop_front().unwrap(),
Operator::Minus,
inputs.pop_front().unwrap(),
)
}
"multiply" => {
ensure_arg_len(2)?;
expr_fn::binary_expr(
inputs.pop_front().unwrap(),
Operator::Multiply,
inputs.pop_front().unwrap(),
)
}
"divide" => {
ensure_arg_len(2)?;
expr_fn::binary_expr(
inputs.pop_front().unwrap(),
Operator::Divide,
inputs.pop_front().unwrap(),
)
}
"modulo" => {
ensure_arg_len(2)?;
expr_fn::binary_expr(
inputs.pop_front().unwrap(),
Operator::Modulo,
inputs.pop_front().unwrap(),
)
}
"and" => {
ensure_arg_len(2)?;
expr_fn::and(inputs.pop_front().unwrap(), inputs.pop_front().unwrap())
}
"or" => {
ensure_arg_len(2)?;
expr_fn::or(inputs.pop_front().unwrap(), inputs.pop_front().unwrap())
}
"like" => {
ensure_arg_len(2)?;
inputs
.pop_front()
.unwrap()
.like(inputs.pop_front().unwrap())
}
"not_like" => {
ensure_arg_len(2)?;
inputs
.pop_front()
.unwrap()
.not_like(inputs.pop_front().unwrap())
}
"is_distinct_from" => {
ensure_arg_len(2)?;
expr_fn::binary_expr(
inputs.pop_front().unwrap(),
Operator::IsDistinctFrom,
inputs.pop_front().unwrap(),
)
}
"is_not_distinct_from" => {
ensure_arg_len(2)?;
expr_fn::binary_expr(
inputs.pop_front().unwrap(),
Operator::IsNotDistinctFrom,
inputs.pop_front().unwrap(),
)
}
"regex_match" => {
ensure_arg_len(2)?;
expr_fn::binary_expr(
inputs.pop_front().unwrap(),
Operator::RegexMatch,
inputs.pop_front().unwrap(),
)
}
"regex_i_match" => {
ensure_arg_len(2)?;
expr_fn::binary_expr(
inputs.pop_front().unwrap(),
Operator::RegexIMatch,
inputs.pop_front().unwrap(),
)
}
"regex_not_match" => {
ensure_arg_len(2)?;
expr_fn::binary_expr(
inputs.pop_front().unwrap(),
Operator::RegexNotMatch,
inputs.pop_front().unwrap(),
)
}
"regex_not_i_match" => {
ensure_arg_len(2)?;
expr_fn::binary_expr(
inputs.pop_front().unwrap(),
Operator::RegexNotIMatch,
inputs.pop_front().unwrap(),
)
}
"bitwise_and" => {
ensure_arg_len(2)?;
expr_fn::binary_expr(
inputs.pop_front().unwrap(),
Operator::BitwiseAnd,
inputs.pop_front().unwrap(),
)
}
"bitwise_or" => {
ensure_arg_len(2)?;
expr_fn::binary_expr(
inputs.pop_front().unwrap(),
Operator::BitwiseOr,
inputs.pop_front().unwrap(),
)
}
// end binary exprs
// start other direct expr, with the same order of DF `Expr`'s definition.
"not" => {
ensure_arg_len(1)?;
inputs.pop_front().unwrap().not()
}
"is_not_null" => {
ensure_arg_len(1)?;
inputs.pop_front().unwrap().is_not_null()
}
"is_null" => {
ensure_arg_len(1)?;
inputs.pop_front().unwrap().is_null()
}
"negative" => {
ensure_arg_len(1)?;
Expr::Negative(Box::new(inputs.pop_front().unwrap()))
}
// skip GetIndexedField, unimplemented.
"between" => {
ensure_arg_len(3)?;
Expr::Between(Between {
expr: Box::new(inputs.pop_front().unwrap()),
negated: false,
low: Box::new(inputs.pop_front().unwrap()),
high: Box::new(inputs.pop_front().unwrap()),
})
}
"not_between" => {
ensure_arg_len(3)?;
Expr::Between(Between {
expr: Box::new(inputs.pop_front().unwrap()),
negated: true,
low: Box::new(inputs.pop_front().unwrap()),
high: Box::new(inputs.pop_front().unwrap()),
})
}
// skip Case, is covered in substrait::SwitchExpression.
// skip Cast and TryCast, is covered in substrait::Cast.
"sort" | "sort_des" => {
ensure_arg_len(1)?;
Expr::Sort(Sort {
expr: Box::new(inputs.pop_front().unwrap()),
asc: false,
nulls_first: false,
})
}
"sort_asc" => {
ensure_arg_len(1)?;
Expr::Sort(Sort {
expr: Box::new(inputs.pop_front().unwrap()),
asc: true,
nulls_first: false,
})
}
// those are datafusion built-in "scalar functions".
"abs"
| "acos"
| "asin"
| "atan"
| "atan2"
| "ceil"
| "cos"
| "exp"
| "floor"
| "ln"
| "log"
| "log10"
| "log2"
| "power"
| "pow"
| "round"
| "signum"
| "sin"
| "sqrt"
| "tan"
| "trunc"
| "coalesce"
| "make_array"
| "ascii"
| "bit_length"
| "btrim"
| "char_length"
| "character_length"
| "concat"
| "concat_ws"
| "chr"
| "current_date"
| "current_time"
| "date_part"
| "datepart"
| "date_trunc"
| "datetrunc"
| "date_bin"
| "initcap"
| "left"
| "length"
| "lower"
| "lpad"
| "ltrim"
| "md5"
| "nullif"
| "octet_length"
| "random"
| "regexp_replace"
| "repeat"
| "replace"
| "reverse"
| "right"
| "rpad"
| "rtrim"
| "sha224"
| "sha256"
| "sha384"
| "sha512"
| "digest"
| "split_part"
| "starts_with"
| "strpos"
| "substr"
| "to_hex"
| "to_timestamp"
| "to_timestamp_millis"
| "to_timestamp_micros"
| "to_timestamp_seconds"
| "now"
| "translate"
| "trim"
| "upper"
| "uuid"
| "regexp_match"
| "struct"
| "from_unixtime"
| "arrow_typeof" => Expr::ScalarFunction {
fun: BuiltinScalarFunction::from_str(fn_name).unwrap(),
args: inputs.into(),
},
// skip ScalarUDF, unimplemented.
// skip AggregateFunction, is covered in substrait::AggregateRel
// skip WindowFunction, is covered in substrait WindowFunction
// skip AggregateUDF, unimplemented.
// skip InList, unimplemented
// skip Wildcard, unimplemented.
// end other direct expr
_ => UnsupportedExprSnafu {
name: format!("scalar function {fn_name}"),
}
.fail()?,
};
Ok(expr)
}
/// Convert DataFusion's `Expr` to substrait's `Expression`
pub fn expression_from_df_expr(
ctx: &mut ConvertorContext,
expr: &Expr,
schema: &Schema,
) -> Result<Expression> {
let expression = match expr {
// Don't merge them with other unsupported expr arms to preserve the ordering.
Expr::Alias(..) => UnsupportedExprSnafu {
name: expr.to_string(),
}
.fail()?,
Expr::Column(column) => {
let field_reference = convert_column(column, schema)?;
Expression {
rex_type: Some(RexType::Selection(Box::new(field_reference))),
}
}
// Don't merge them with other unsupported expr arms to preserve the ordering.
Expr::ScalarVariable(..) => UnsupportedExprSnafu {
name: expr.to_string(),
}
.fail()?,
Expr::Literal(v) => {
let t = scalar_value_as_literal_type(v)?;
let l = Literal {
nullable: true,
type_variation_reference: 0,
literal_type: Some(t),
};
Expression {
rex_type: Some(RexType::Literal(l)),
}
}
Expr::BinaryExpr(BinaryExpr { left, op, right }) => {
let left = expression_from_df_expr(ctx, left, schema)?;
let right = expression_from_df_expr(ctx, right, schema)?;
let arguments = utils::expression_to_argument(vec![left, right]);
let op_name = utils::name_df_operator(op);
let function_reference = ctx.register_scalar_fn(op_name);
utils::build_scalar_function_expression(function_reference, arguments)
}
Expr::Not(e) => {
let arg = expression_from_df_expr(ctx, e, schema)?;
let arguments = utils::expression_to_argument(vec![arg]);
let op_name = "not";
let function_reference = ctx.register_scalar_fn(op_name);
utils::build_scalar_function_expression(function_reference, arguments)
}
Expr::IsNotNull(e) => {
let arg = expression_from_df_expr(ctx, e, schema)?;
let arguments = utils::expression_to_argument(vec![arg]);
let op_name = "is_not_null";
let function_reference = ctx.register_scalar_fn(op_name);
utils::build_scalar_function_expression(function_reference, arguments)
}
Expr::IsNull(e) => {
let arg = expression_from_df_expr(ctx, e, schema)?;
let arguments = utils::expression_to_argument(vec![arg]);
let op_name = "is_null";
let function_reference = ctx.register_scalar_fn(op_name);
utils::build_scalar_function_expression(function_reference, arguments)
}
Expr::Negative(e) => {
let arg = expression_from_df_expr(ctx, e, schema)?;
let arguments = utils::expression_to_argument(vec![arg]);
let op_name = "negative";
let function_reference = ctx.register_scalar_fn(op_name);
utils::build_scalar_function_expression(function_reference, arguments)
}
// Don't merge them with other unsupported expr arms to preserve the ordering.
Expr::GetIndexedField { .. } => UnsupportedExprSnafu {
name: expr.to_string(),
}
.fail()?,
Expr::Between(Between {
expr,
negated,
low,
high,
}) => {
let expr = expression_from_df_expr(ctx, expr, schema)?;
let low = expression_from_df_expr(ctx, low, schema)?;
let high = expression_from_df_expr(ctx, high, schema)?;
let arguments = utils::expression_to_argument(vec![expr, low, high]);
let op_name = if *negated { "not_between" } else { "between" };
let function_reference = ctx.register_scalar_fn(op_name);
utils::build_scalar_function_expression(function_reference, arguments)
}
// Don't merge them with other unsupported expr arms to preserve the ordering.
Expr::Case { .. } | Expr::Cast { .. } | Expr::TryCast { .. } => UnsupportedExprSnafu {
name: expr.to_string(),
}
.fail()?,
Expr::Sort(Sort {
expr,
asc,
nulls_first: _,
}) => {
let expr = expression_from_df_expr(ctx, expr, schema)?;
let arguments = utils::expression_to_argument(vec![expr]);
let op_name = if *asc { "sort_asc" } else { "sort_des" };
let function_reference = ctx.register_scalar_fn(op_name);
utils::build_scalar_function_expression(function_reference, arguments)
}
Expr::ScalarFunction { fun, args } => {
let arguments = utils::expression_to_argument(
args.iter()
.map(|e| expression_from_df_expr(ctx, e, schema))
.collect::<Result<Vec<_>>>()?,
);
let op_name = utils::name_builtin_scalar_function(fun);
let function_reference = ctx.register_scalar_fn(op_name);
utils::build_scalar_function_expression(function_reference, arguments)
}
// Don't merge them with other unsupported expr arms to preserve the ordering.
Expr::ScalarUDF { .. }
| Expr::AggregateFunction { .. }
| Expr::WindowFunction { .. }
| Expr::AggregateUDF { .. }
| Expr::InList { .. }
| Expr::Wildcard
| Expr::Like(_)
| Expr::ILike(_)
| Expr::SimilarTo(_)
| Expr::IsTrue(_)
| Expr::IsFalse(_)
| Expr::IsUnknown(_)
| Expr::IsNotTrue(_)
| Expr::IsNotFalse(_)
| Expr::IsNotUnknown(_)
| Expr::Exists { .. }
| Expr::InSubquery { .. }
| Expr::ScalarSubquery(..)
| Expr::Placeholder { .. }
| Expr::QualifiedWildcard { .. } => todo!(),
Expr::GroupingSet(_) | Expr::OuterReferenceColumn(_, _) => UnsupportedExprSnafu {
name: expr.to_string(),
}
.fail()?,
};
Ok(expression)
}
/// Convert DataFusion's `Column` expr into substrait's `FieldReference` -
/// `DirectReference` - `StructField`.
pub fn convert_column(column: &Column, schema: &Schema) -> Result<FieldReference> {
let column_name = &column.name;
let field_index =
schema
.column_index_by_name(column_name)
.with_context(|| MissingFieldSnafu {
field: format!("{column:?}"),
plan: format!("schema: {schema:?}"),
})?;
Ok(FieldReference {
reference_type: Some(FieldReferenceType::DirectReference(ReferenceSegment {
reference_type: Some(SegReferenceType::StructField(Box::new(StructField {
field: field_index as _,
child: None,
}))),
})),
root_type: None,
})
}
/// Some utils special for this `DataFusion::Expr` and `Substrait::Expression` conversion.
mod utils {
use datafusion_expr::{BuiltinScalarFunction, Operator};
use substrait_proto::proto::expression::{RexType, ScalarFunction};
use substrait_proto::proto::function_argument::ArgType;
use substrait_proto::proto::{Expression, FunctionArgument};
pub(crate) fn name_df_operator(op: &Operator) -> &str {
match op {
Operator::Eq => "equal",
Operator::NotEq => "not_equal",
Operator::Lt => "lt",
Operator::LtEq => "lte",
Operator::Gt => "gt",
Operator::GtEq => "gte",
Operator::Plus => "plus",
Operator::Minus => "minus",
Operator::Multiply => "multiply",
Operator::Divide => "divide",
Operator::Modulo => "modulo",
Operator::And => "and",
Operator::Or => "or",
Operator::IsDistinctFrom => "is_distinct_from",
Operator::IsNotDistinctFrom => "is_not_distinct_from",
Operator::RegexMatch => "regex_match",
Operator::RegexIMatch => "regex_i_match",
Operator::RegexNotMatch => "regex_not_match",
Operator::RegexNotIMatch => "regex_not_i_match",
Operator::BitwiseAnd => "bitwise_and",
Operator::BitwiseOr => "bitwise_or",
Operator::BitwiseXor => "bitwise_xor",
Operator::BitwiseShiftRight => "bitwise_shift_right",
Operator::BitwiseShiftLeft => "bitwise_shift_left",
Operator::StringConcat => "string_concat",
}
}
/// Convert list of [Expression] to [FunctionArgument] vector.
pub(crate) fn expression_to_argument<I: IntoIterator<Item = Expression>>(
expressions: I,
) -> Vec<FunctionArgument> {
expressions
.into_iter()
.map(|expr| FunctionArgument {
arg_type: Some(ArgType::Value(expr)),
})
.collect()
}
/// Convenient builder for [Expression]
pub(crate) fn build_scalar_function_expression(
function_reference: u32,
arguments: Vec<FunctionArgument>,
) -> Expression {
Expression {
rex_type: Some(RexType::ScalarFunction(ScalarFunction {
function_reference,
arguments,
output_type: None,
..Default::default()
})),
}
}
pub(crate) fn name_builtin_scalar_function(fun: &BuiltinScalarFunction) -> &str {
match fun {
BuiltinScalarFunction::Abs => "abs",
BuiltinScalarFunction::Acos => "acos",
BuiltinScalarFunction::Asin => "asin",
BuiltinScalarFunction::Atan => "atan",
BuiltinScalarFunction::Ceil => "ceil",
BuiltinScalarFunction::Cos => "cos",
BuiltinScalarFunction::Digest => "digest",
BuiltinScalarFunction::Exp => "exp",
BuiltinScalarFunction::Floor => "floor",
BuiltinScalarFunction::Ln => "ln",
BuiltinScalarFunction::Log => "log",
BuiltinScalarFunction::Log10 => "log10",
BuiltinScalarFunction::Log2 => "log2",
BuiltinScalarFunction::Round => "round",
BuiltinScalarFunction::Signum => "signum",
BuiltinScalarFunction::Sin => "sin",
BuiltinScalarFunction::Sqrt => "sqrt",
BuiltinScalarFunction::Tan => "tan",
BuiltinScalarFunction::Trunc => "trunc",
BuiltinScalarFunction::Ascii => "ascii",
BuiltinScalarFunction::BitLength => "bit_length",
BuiltinScalarFunction::Btrim => "btrim",
BuiltinScalarFunction::CharacterLength => "character_length",
BuiltinScalarFunction::Chr => "chr",
BuiltinScalarFunction::Concat => "concat",
BuiltinScalarFunction::ConcatWithSeparator => "concat_ws",
BuiltinScalarFunction::DatePart => "date_part",
BuiltinScalarFunction::DateTrunc => "date_trunc",
BuiltinScalarFunction::InitCap => "initcap",
BuiltinScalarFunction::Left => "left",
BuiltinScalarFunction::Lpad => "lpad",
BuiltinScalarFunction::Lower => "lower",
BuiltinScalarFunction::Ltrim => "ltrim",
BuiltinScalarFunction::MD5 => "md5",
BuiltinScalarFunction::NullIf => "nullif",
BuiltinScalarFunction::OctetLength => "octet_length",
BuiltinScalarFunction::Random => "random",
BuiltinScalarFunction::RegexpReplace => "regexp_replace",
BuiltinScalarFunction::Repeat => "repeat",
BuiltinScalarFunction::Replace => "replace",
BuiltinScalarFunction::Reverse => "reverse",
BuiltinScalarFunction::Right => "right",
BuiltinScalarFunction::Rpad => "rpad",
BuiltinScalarFunction::Rtrim => "rtrim",
BuiltinScalarFunction::SHA224 => "sha224",
BuiltinScalarFunction::SHA256 => "sha256",
BuiltinScalarFunction::SHA384 => "sha384",
BuiltinScalarFunction::SHA512 => "sha512",
BuiltinScalarFunction::SplitPart => "split_part",
BuiltinScalarFunction::StartsWith => "starts_with",
BuiltinScalarFunction::Strpos => "strpos",
BuiltinScalarFunction::Substr => "substr",
BuiltinScalarFunction::ToHex => "to_hex",
BuiltinScalarFunction::ToTimestamp => "to_timestamp",
BuiltinScalarFunction::ToTimestampMillis => "to_timestamp_millis",
BuiltinScalarFunction::ToTimestampMicros => "to_timestamp_macros",
BuiltinScalarFunction::ToTimestampSeconds => "to_timestamp_seconds",
BuiltinScalarFunction::Now => "now",
BuiltinScalarFunction::Translate => "translate",
BuiltinScalarFunction::Trim => "trim",
BuiltinScalarFunction::Upper => "upper",
BuiltinScalarFunction::RegexpMatch => "regexp_match",
BuiltinScalarFunction::Atan2 => "atan2",
BuiltinScalarFunction::Coalesce => "coalesce",
BuiltinScalarFunction::Power => "power",
BuiltinScalarFunction::MakeArray => "make_array",
BuiltinScalarFunction::DateBin => "date_bin",
BuiltinScalarFunction::FromUnixtime => "from_unixtime",
BuiltinScalarFunction::CurrentDate => "current_date",
BuiltinScalarFunction::CurrentTime => "current_time",
BuiltinScalarFunction::Uuid => "uuid",
BuiltinScalarFunction::Struct => "struct",
BuiltinScalarFunction::ArrowTypeof => "arrow_type_of",
BuiltinScalarFunction::Acosh => "acosh",
BuiltinScalarFunction::Asinh => "asinh",
BuiltinScalarFunction::Atanh => "atanh",
BuiltinScalarFunction::Cbrt => "cbrt",
BuiltinScalarFunction::Cosh => "cosh",
BuiltinScalarFunction::Pi => "pi",
BuiltinScalarFunction::Sinh => "sinh",
BuiltinScalarFunction::Tanh => "tanh",
}
}
}
#[cfg(test)]
mod test {
use datatypes::schema::ColumnSchema;
use super::*;
#[test]
fn expr_round_trip() {
let expr = expr_fn::and(
expr_fn::col("column_a").lt_eq(expr_fn::col("column_b")),
expr_fn::col("column_a").gt(expr_fn::col("column_b")),
);
let schema = Schema::new(vec![
ColumnSchema::new(
"column_a",
datatypes::data_type::ConcreteDataType::int64_datatype(),
true,
),
ColumnSchema::new(
"column_b",
datatypes::data_type::ConcreteDataType::float64_datatype(),
true,
),
]);
let mut ctx = ConvertorContext::default();
let substrait_expr = expression_from_df_expr(&mut ctx, &expr, &schema).unwrap();
let converted_expr = to_df_expr(&ctx, substrait_expr, &schema).unwrap();
assert_eq!(expr, converted_expr);
}
}

View File

@@ -1,534 +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::sync::Arc;
use async_recursion::async_recursion;
use async_trait::async_trait;
use bytes::{Buf, Bytes};
use catalog::table_source::DfTableSourceProvider;
use catalog::CatalogManagerRef;
use common_catalog::format_full_table_name;
use common_telemetry::debug;
use datafusion::arrow::datatypes::SchemaRef as ArrowSchemaRef;
use datafusion::catalog::catalog::CatalogList;
use datafusion::common::{DFField, DFSchema};
use datafusion::datasource::DefaultTableSource;
use datafusion::physical_plan::project_schema;
use datafusion::sql::TableReference;
use datafusion_expr::{Filter, LogicalPlan, TableScan};
use session::context::QueryContext;
use snafu::{ensure, OptionExt, ResultExt};
use substrait_proto::proto::expression::mask_expression::{StructItem, StructSelect};
use substrait_proto::proto::expression::MaskExpression;
use substrait_proto::proto::extensions::simple_extension_declaration::MappingType;
use substrait_proto::proto::plan_rel::RelType as PlanRelType;
use substrait_proto::proto::read_rel::{NamedTable, ReadType};
use substrait_proto::proto::rel::RelType;
use substrait_proto::proto::{FilterRel, Plan, PlanRel, ReadRel, Rel};
use table::table::adapter::DfTableProviderAdapter;
use crate::context::ConvertorContext;
use crate::df_expr::{expression_from_df_expr, to_df_expr};
use crate::error::{
self, DFInternalSnafu, EmptyPlanSnafu, Error, InvalidParametersSnafu, MissingFieldSnafu,
ResolveTableSnafu, SchemaNotMatchSnafu, UnknownPlanSnafu, UnsupportedExprSnafu,
UnsupportedPlanSnafu,
};
use crate::schema::{from_schema, to_schema};
use crate::SubstraitPlan;
pub struct DFLogicalSubstraitConvertorDeprecated;
#[async_trait]
impl SubstraitPlan for DFLogicalSubstraitConvertorDeprecated {
type Error = Error;
type Plan = LogicalPlan;
async fn decode<B: Buf + Send>(
&self,
_message: B,
_catalog_list: Arc<dyn CatalogList>,
) -> Result<Self::Plan, Self::Error> {
unimplemented!()
}
fn encode(&self, plan: Self::Plan) -> Result<Bytes, Self::Error> {
unimplemented!()
}
}
impl DFLogicalSubstraitConvertorDeprecated {
async fn convert_plan(
&self,
mut plan: Plan,
catalog_manager: CatalogManagerRef,
) -> Result<LogicalPlan, Error> {
// prepare convertor context
let mut ctx = ConvertorContext::default();
for simple_ext in plan.extensions {
if let Some(MappingType::ExtensionFunction(function_extension)) =
simple_ext.mapping_type
{
ctx.register_scalar_with_anchor(
function_extension.name,
function_extension.function_anchor,
);
} else {
debug!("Encounter unsupported substrait extension {:?}", simple_ext);
}
}
// extract rel
let rel = if let Some(PlanRel { rel_type }) = plan.relations.pop()
&& let Some(PlanRelType::Rel(rel)) = rel_type {
rel
} else {
UnsupportedPlanSnafu {
name: "Emply or non-Rel relation",
}
.fail()?
};
// TODO(LFC): Create table provider from outside, respect "disallow_cross_schema_query" option in query engine state.
let mut table_provider =
DfTableSourceProvider::new(catalog_manager, false, &QueryContext::new());
self.rel_to_logical_plan(&mut ctx, Box::new(rel), &mut table_provider)
.await
}
#[async_recursion]
async fn rel_to_logical_plan(
&self,
ctx: &mut ConvertorContext,
rel: Box<Rel>,
table_provider: &mut DfTableSourceProvider,
) -> Result<LogicalPlan, Error> {
let rel_type = rel.rel_type.context(EmptyPlanSnafu)?;
// build logical plan
let logical_plan = match rel_type {
RelType::Read(read_rel) => self.convert_read_rel(ctx, read_rel, table_provider).await?,
RelType::Filter(filter) => {
let FilterRel {
common: _,
input,
condition,
advanced_extension: _,
} = *filter;
let input = input.context(MissingFieldSnafu {
field: "input",
plan: "Filter",
})?;
let input = Arc::new(self.rel_to_logical_plan(ctx, input, table_provider).await?);
let condition = condition.context(MissingFieldSnafu {
field: "condition",
plan: "Filter",
})?;
let schema = ctx.df_schema().context(InvalidParametersSnafu {
reason: "the underlying TableScan plan should have included a table schema",
})?;
let schema = schema
.clone()
.try_into()
.context(error::ConvertDfSchemaSnafu)?;
let predicate = to_df_expr(ctx, *condition, &schema)?;
LogicalPlan::Filter(Filter::try_new(predicate, input).context(DFInternalSnafu)?)
}
RelType::Fetch(_fetch_rel) => UnsupportedPlanSnafu {
name: "Fetch Relation",
}
.fail()?,
RelType::Aggregate(_aggr_rel) => UnsupportedPlanSnafu {
name: "Fetch Relation",
}
.fail()?,
RelType::Sort(_sort_rel) => UnsupportedPlanSnafu {
name: "Sort Relation",
}
.fail()?,
RelType::Join(_join_rel) => UnsupportedPlanSnafu {
name: "Join Relation",
}
.fail()?,
RelType::Project(_project_rel) => UnsupportedPlanSnafu {
name: "Project Relation",
}
.fail()?,
RelType::Set(_set_rel) => UnsupportedPlanSnafu {
name: "Set Relation",
}
.fail()?,
RelType::ExtensionSingle(_ext_single_rel) => UnsupportedPlanSnafu {
name: "Extension Single Relation",
}
.fail()?,
RelType::ExtensionMulti(_ext_multi_rel) => UnsupportedPlanSnafu {
name: "Extension Multi Relation",
}
.fail()?,
RelType::ExtensionLeaf(_ext_leaf_rel) => UnsupportedPlanSnafu {
name: "Extension Leaf Relation",
}
.fail()?,
RelType::Cross(_cross_rel) => UnsupportedPlanSnafu {
name: "Cross Relation",
}
.fail()?,
RelType::HashJoin(_) => UnsupportedPlanSnafu {
name: "Cross Relation",
}
.fail()?,
RelType::MergeJoin(_) => UnsupportedPlanSnafu {
name: "Cross Relation",
}
.fail()?,
};
Ok(logical_plan)
}
async fn convert_read_rel(
&self,
ctx: &mut ConvertorContext,
read_rel: Box<ReadRel>,
table_provider: &mut DfTableSourceProvider,
) -> Result<LogicalPlan, Error> {
// Extract the catalog, schema and table name from NamedTable. Assume the first three are those names.
let read_type = read_rel.read_type.context(MissingFieldSnafu {
field: "read_type",
plan: "Read",
})?;
let (table_name, schema_name, catalog_name) = match read_type {
ReadType::NamedTable(mut named_table) => {
ensure!(
named_table.names.len() == 3,
InvalidParametersSnafu {
reason:
"NamedTable should contains three names for catalog, schema and table",
}
);
(
named_table.names.pop().unwrap(),
named_table.names.pop().unwrap(),
named_table.names.pop().unwrap(),
)
}
ReadType::VirtualTable(_) | ReadType::LocalFiles(_) | ReadType::ExtensionTable(_) => {
UnsupportedExprSnafu {
name: "Non-NamedTable Read",
}
.fail()?
}
};
// Get projection indices
let projection = read_rel
.projection
.map(|mask_expr| self.convert_mask_expression(mask_expr));
let table_ref = TableReference::full(
catalog_name.clone(),
schema_name.clone(),
table_name.clone(),
);
let adapter = table_provider
.resolve_table(table_ref.clone())
.await
.with_context(|_| ResolveTableSnafu {
table_name: format_full_table_name(&catalog_name, &schema_name, &table_name),
})?;
// Get schema directly from the table, and compare it with the schema retrieved from substrait proto.
let stored_schema = adapter.schema();
let retrieved_schema = to_schema(read_rel.base_schema.unwrap_or_default())?;
let retrieved_arrow_schema = retrieved_schema.arrow_schema();
ensure!(
same_schema_without_metadata(&stored_schema, retrieved_arrow_schema),
SchemaNotMatchSnafu {
substrait_schema: retrieved_arrow_schema.clone(),
storage_schema: stored_schema
}
);
// Convert filter
let filters = if let Some(filter) = read_rel.filter {
vec![to_df_expr(ctx, *filter, &retrieved_schema)?]
} else {
vec![]
};
// Calculate the projected schema
let projected_schema = Arc::new(
project_schema(&stored_schema, projection.as_ref())
.and_then(|x| {
DFSchema::new_with_metadata(
x.fields()
.iter()
.map(|f| DFField::from_qualified(table_ref.clone(), f.clone()))
.collect(),
x.metadata().clone(),
)
})
.context(DFInternalSnafu)?,
);
ctx.set_df_schema(projected_schema.clone());
// TODO(ruihang): Support limit(fetch)
Ok(LogicalPlan::TableScan(TableScan {
table_name: table_ref,
source: adapter,
projection,
projected_schema,
filters,
fetch: None,
}))
}
fn convert_mask_expression(&self, mask_expression: MaskExpression) -> Vec<usize> {
mask_expression
.select
.unwrap_or_default()
.struct_items
.into_iter()
.map(|select| select.field as _)
.collect()
}
}
impl DFLogicalSubstraitConvertorDeprecated {
fn logical_plan_to_rel(
&self,
ctx: &mut ConvertorContext,
plan: Arc<LogicalPlan>,
) -> Result<Rel, Error> {
Ok(match &*plan {
LogicalPlan::Projection(_) => UnsupportedPlanSnafu {
name: "DataFusion Logical Projection",
}
.fail()?,
LogicalPlan::Filter(filter) => {
let input = Some(Box::new(
self.logical_plan_to_rel(ctx, filter.input.clone())?,
));
let schema = plan
.schema()
.clone()
.try_into()
.context(error::ConvertDfSchemaSnafu)?;
let condition = Some(Box::new(expression_from_df_expr(
ctx,
&filter.predicate,
&schema,
)?));
let rel = FilterRel {
common: None,
input,
condition,
advanced_extension: None,
};
Rel {
rel_type: Some(RelType::Filter(Box::new(rel))),
}
}
LogicalPlan::Window(_) => UnsupportedPlanSnafu {
name: "DataFusion Logical Window",
}
.fail()?,
LogicalPlan::Aggregate(_) => UnsupportedPlanSnafu {
name: "DataFusion Logical Aggregate",
}
.fail()?,
LogicalPlan::Sort(_) => UnsupportedPlanSnafu {
name: "DataFusion Logical Sort",
}
.fail()?,
LogicalPlan::Join(_) => UnsupportedPlanSnafu {
name: "DataFusion Logical Join",
}
.fail()?,
LogicalPlan::CrossJoin(_) => UnsupportedPlanSnafu {
name: "DataFusion Logical CrossJoin",
}
.fail()?,
LogicalPlan::Repartition(_) => UnsupportedPlanSnafu {
name: "DataFusion Logical Repartition",
}
.fail()?,
LogicalPlan::Union(_) => UnsupportedPlanSnafu {
name: "DataFusion Logical Union",
}
.fail()?,
LogicalPlan::TableScan(table_scan) => {
let read_rel = self.convert_table_scan_plan(ctx, table_scan)?;
Rel {
rel_type: Some(RelType::Read(Box::new(read_rel))),
}
}
LogicalPlan::EmptyRelation(_) => UnsupportedPlanSnafu {
name: "DataFusion Logical EmptyRelation",
}
.fail()?,
LogicalPlan::Limit(_) => UnsupportedPlanSnafu {
name: "DataFusion Logical Limit",
}
.fail()?,
LogicalPlan::Subquery(_)
| LogicalPlan::SubqueryAlias(_)
| LogicalPlan::CreateView(_)
| LogicalPlan::CreateCatalogSchema(_)
| LogicalPlan::CreateCatalog(_)
| LogicalPlan::DropView(_)
| LogicalPlan::Distinct(_)
| LogicalPlan::CreateExternalTable(_)
| LogicalPlan::CreateMemoryTable(_)
| LogicalPlan::DropTable(_)
| LogicalPlan::Values(_)
| LogicalPlan::Explain(_)
| LogicalPlan::Analyze(_)
| LogicalPlan::Extension(_)
| LogicalPlan::Prepare(_)
| LogicalPlan::Dml(_)
| LogicalPlan::DescribeTable(_)
| LogicalPlan::Unnest(_)
| LogicalPlan::Statement(_) => InvalidParametersSnafu {
reason: format!(
"Trying to convert DDL/DML plan to substrait proto, plan: {plan:?}",
),
}
.fail()?,
})
}
fn convert_df_plan(&self, plan: LogicalPlan) -> Result<Plan, Error> {
let mut ctx = ConvertorContext::default();
let rel = self.logical_plan_to_rel(&mut ctx, Arc::new(plan))?;
// convert extension
let extensions = ctx.generate_function_extension();
// assemble PlanRel
let plan_rel = PlanRel {
rel_type: Some(PlanRelType::Rel(rel)),
};
Ok(Plan {
extension_uris: vec![],
extensions,
relations: vec![plan_rel],
advanced_extensions: None,
expected_type_urls: vec![],
..Default::default()
})
}
pub fn convert_table_scan_plan(
&self,
ctx: &mut ConvertorContext,
table_scan: &TableScan,
) -> Result<ReadRel, Error> {
let provider = table_scan
.source
.as_any()
.downcast_ref::<DefaultTableSource>()
.context(UnknownPlanSnafu)?
.table_provider
.as_any()
.downcast_ref::<DfTableProviderAdapter>()
.context(UnknownPlanSnafu)?;
let table_info = provider.table().table_info();
// assemble NamedTable and ReadType
let catalog_name = table_info.catalog_name.clone();
let schema_name = table_info.schema_name.clone();
let table_name = table_info.name.clone();
let named_table = NamedTable {
names: vec![catalog_name, schema_name, table_name],
advanced_extension: None,
};
let read_type = ReadType::NamedTable(named_table);
// assemble projection
let projection = table_scan
.projection
.as_ref()
.map(|x| self.convert_schema_projection(x));
// assemble base (unprojected) schema using Table's schema.
let base_schema = from_schema(&provider.table().schema())?;
// make conjunction over a list of filters and convert the result to substrait
let filter = if let Some(conjunction) = table_scan
.filters
.iter()
.cloned()
.reduce(|accum, expr| accum.and(expr))
{
Some(Box::new(expression_from_df_expr(
ctx,
&conjunction,
&provider.table().schema(),
)?))
} else {
None
};
let read_rel = ReadRel {
common: None,
base_schema: Some(base_schema),
filter,
projection,
advanced_extension: None,
read_type: Some(read_type),
..Default::default()
};
Ok(read_rel)
}
/// Convert a index-based schema projection to substrait's [MaskExpression].
fn convert_schema_projection(&self, projections: &[usize]) -> MaskExpression {
let struct_items = projections
.iter()
.map(|index| StructItem {
field: *index as i32,
child: None,
})
.collect();
MaskExpression {
select: Some(StructSelect { struct_items }),
// TODO(ruihang): this field is unspecified
maintain_singular_struct: true,
}
}
}
fn same_schema_without_metadata(lhs: &ArrowSchemaRef, rhs: &ArrowSchemaRef) -> bool {
lhs.fields.len() == rhs.fields.len()
&& lhs.fields.iter().zip(rhs.fields.iter()).all(|(x, y)| {
x.name() == y.name()
&& x.data_type() == y.data_type()
&& x.is_nullable() == y.is_nullable()
})
}

View File

@@ -16,8 +16,11 @@ use std::sync::Arc;
use async_trait::async_trait;
use bytes::{Buf, Bytes, BytesMut};
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
use datafusion::catalog::catalog::CatalogList;
use datafusion::prelude::SessionContext;
use datafusion::execution::context::SessionState;
use datafusion::execution::runtime_env::RuntimeEnv;
use datafusion::prelude::{SessionConfig, SessionContext};
use datafusion_expr::LogicalPlan;
use datafusion_substrait::logical_plan::consumer::from_substrait_plan;
use datafusion_substrait::logical_plan::producer::to_substrait_plan;
@@ -41,9 +44,12 @@ impl SubstraitPlan for DFLogicalSubstraitConvertor {
message: B,
catalog_list: Arc<dyn CatalogList>,
) -> Result<Self::Plan, Self::Error> {
let mut context = SessionContext::new();
let plan = Plan::decode(message).context(DecodeRelSnafu)?;
let state_config = SessionConfig::new()
.with_default_catalog_and_schema(DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME);
let state = SessionState::with_config_rt(state_config, Arc::new(RuntimeEnv::default()));
let mut context = SessionContext::with_state(state);
context.register_catalog_list(catalog_list);
let plan = Plan::decode(message).context(DecodeRelSnafu)?;
let df_plan = from_substrait_plan(&mut context, &plan)
.await
.context(DecodeDfPlanSnafu)?;
@@ -52,8 +58,9 @@ impl SubstraitPlan for DFLogicalSubstraitConvertor {
fn encode(&self, plan: Self::Plan) -> Result<Bytes, Self::Error> {
let mut buf = BytesMut::new();
let context = SessionContext::new();
let substrait_plan = to_substrait_plan(&plan).context(EncodeDfPlanSnafu)?;
let substrait_plan = to_substrait_plan(&plan, &context).context(EncodeDfPlanSnafu)?;
substrait_plan.encode(&mut buf).context(EncodeRelSnafu)?;
Ok(buf.freeze())

View File

@@ -15,14 +15,8 @@
#![feature(let_chains)]
#![feature(trait_upcasting)]
mod context;
mod df_expr;
#[allow(unused)]
mod df_logical;
mod df_substrait;
pub mod error;
mod schema;
mod types;
use std::sync::Arc;

View File

@@ -1,111 +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 datatypes::schema::{ColumnSchema, Schema};
use substrait_proto::proto::r#type::{Nullability, Struct as SubstraitStruct};
use substrait_proto::proto::NamedStruct;
use crate::error::Result;
use crate::types::{from_concrete_type, to_concrete_type};
pub fn to_schema(named_struct: NamedStruct) -> Result<Schema> {
if named_struct.r#struct.is_none() {
return Ok(Schema::new(vec![]));
}
let column_schemas = named_struct
.r#struct
.unwrap()
.types
.into_iter()
.zip(named_struct.names.into_iter())
.map(|(ty, name)| {
let (concrete_type, is_nullable) = to_concrete_type(&ty)?;
let column_schema = ColumnSchema::new(name, concrete_type, is_nullable);
Ok(column_schema)
})
.collect::<Result<_>>()?;
Ok(Schema::new(column_schemas))
}
pub fn from_schema(schema: &Schema) -> Result<NamedStruct> {
let mut names = Vec::with_capacity(schema.num_columns());
let mut types = Vec::with_capacity(schema.num_columns());
for column_schema in schema.column_schemas() {
names.push(column_schema.name.clone());
let substrait_type = from_concrete_type(
column_schema.data_type.clone(),
Some(column_schema.is_nullable()),
)?;
types.push(substrait_type);
}
// TODO(ruihang): `type_variation_reference` and `nullability` are unspecified.
let substrait_struct = SubstraitStruct {
types,
type_variation_reference: 0,
nullability: Nullability::Unspecified as _,
};
Ok(NamedStruct {
names,
r#struct: Some(substrait_struct),
})
}
#[cfg(test)]
pub(crate) mod test {
use datatypes::prelude::{ConcreteDataType, DataType};
use super::*;
pub(crate) fn supported_types() -> Vec<ColumnSchema> {
[
ConcreteDataType::null_datatype(),
ConcreteDataType::boolean_datatype(),
ConcreteDataType::int8_datatype(),
ConcreteDataType::int16_datatype(),
ConcreteDataType::int32_datatype(),
ConcreteDataType::int64_datatype(),
ConcreteDataType::uint8_datatype(),
ConcreteDataType::uint16_datatype(),
ConcreteDataType::uint32_datatype(),
ConcreteDataType::uint64_datatype(),
ConcreteDataType::float32_datatype(),
ConcreteDataType::float64_datatype(),
ConcreteDataType::binary_datatype(),
ConcreteDataType::string_datatype(),
ConcreteDataType::date_datatype(),
ConcreteDataType::timestamp_datatype(Default::default()),
// TODO(ruihang): DateTime and List type are not supported now
]
.into_iter()
.enumerate()
.map(|(ordinal, ty)| ColumnSchema::new(ty.name().to_string(), ty, ordinal % 2 == 0))
.collect()
}
#[test]
fn supported_types_round_trip() {
let column_schemas = supported_types();
let schema = Schema::new(column_schemas);
let named_struct = from_schema(&schema).unwrap();
let converted_schema = to_schema(named_struct).unwrap();
assert_eq!(schema, converted_schema);
}
}

View File

@@ -1,225 +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.
//! Methods that perform conversion between Substrait's type ([Type](SType)) and GreptimeDB's type ([ConcreteDataType]).
//!
//! Substrait use [type variation](https://substrait.io/types/type_variations/) to express different "logical types".
//! Current we only have variations on integer types. Variation 0 (system preferred) are the same with base types, which
//! are signed integer (i.e. I8 -> [i8]), and Variation 1 stands for unsigned integer (i.e. I8 -> [u8]).
use datafusion::scalar::ScalarValue;
use datatypes::prelude::ConcreteDataType;
use datatypes::types::TimestampType;
use substrait_proto::proto::expression::literal::LiteralType;
use substrait_proto::proto::r#type::{self as s_type, Kind, Nullability};
use substrait_proto::proto::{Type as SType, Type};
use crate::error::{self, Result, UnsupportedConcreteTypeSnafu, UnsupportedSubstraitTypeSnafu};
macro_rules! substrait_kind {
($desc:ident, $concrete_ty:ident) => {{
let nullable = $desc.nullability() == Nullability::Nullable;
let ty = ConcreteDataType::$concrete_ty();
Ok((ty, nullable))
}};
($desc:ident, $concrete_ty:expr) => {{
let nullable = $desc.nullability() == Nullability::Nullable;
Ok(($concrete_ty, nullable))
}};
($desc:ident, $concrete_ty_0:ident, $concrete_ty_1:ident) => {{
let nullable = $desc.nullability() == Nullability::Nullable;
let ty = match $desc.type_variation_reference {
0 => ConcreteDataType::$concrete_ty_0(),
1 => ConcreteDataType::$concrete_ty_1(),
_ => UnsupportedSubstraitTypeSnafu {
ty: format!("{:?}", $desc),
}
.fail()?,
};
Ok((ty, nullable))
}};
}
/// Convert Substrait [Type](SType) to GreptimeDB's [ConcreteDataType]. The bool in return
/// tuple is the nullability identifier.
pub fn to_concrete_type(ty: &SType) -> Result<(ConcreteDataType, bool)> {
if ty.kind.is_none() {
return Ok((ConcreteDataType::null_datatype(), true));
}
let kind = ty.kind.as_ref().unwrap();
match kind {
Kind::Bool(desc) => substrait_kind!(desc, boolean_datatype),
Kind::I8(desc) => substrait_kind!(desc, int8_datatype, uint8_datatype),
Kind::I16(desc) => substrait_kind!(desc, int16_datatype, uint16_datatype),
Kind::I32(desc) => substrait_kind!(desc, int32_datatype, uint32_datatype),
Kind::I64(desc) => substrait_kind!(desc, int64_datatype, uint64_datatype),
Kind::Fp32(desc) => substrait_kind!(desc, float32_datatype),
Kind::Fp64(desc) => substrait_kind!(desc, float64_datatype),
Kind::String(desc) => substrait_kind!(desc, string_datatype),
Kind::Binary(desc) => substrait_kind!(desc, binary_datatype),
Kind::Timestamp(desc) => substrait_kind!(
desc,
ConcreteDataType::timestamp_datatype(
TimestampType::try_from(desc.type_variation_reference as u64)
.map_err(|_| UnsupportedSubstraitTypeSnafu {
ty: format!("{kind:?}")
}
.build())?
.unit()
)
),
Kind::Date(desc) => substrait_kind!(desc, date_datatype),
Kind::Time(_)
| Kind::IntervalYear(_)
| Kind::IntervalDay(_)
| Kind::TimestampTz(_)
| Kind::Uuid(_)
| Kind::FixedChar(_)
| Kind::Varchar(_)
| Kind::FixedBinary(_)
| Kind::Decimal(_)
| Kind::Struct(_)
| Kind::List(_)
| Kind::Map(_)
| Kind::UserDefined(_)
| Kind::UserDefinedTypeReference(_) => UnsupportedSubstraitTypeSnafu {
ty: format!("{kind:?}"),
}
.fail(),
}
}
macro_rules! build_substrait_kind {
($kind:ident,$s_type:ident,$nullable:ident,$variation:expr) => {{
let nullability = match $nullable {
Some(true) => Nullability::Nullable,
Some(false) => Nullability::Required,
None => Nullability::Unspecified,
} as _;
Some(Kind::$kind(s_type::$s_type {
type_variation_reference: $variation,
nullability,
}))
}};
}
/// Convert GreptimeDB's [ConcreteDataType] to Substrait [Type](SType).
///
/// Refer to [mod level documentation](super::types) for more information about type variation.
pub fn from_concrete_type(ty: ConcreteDataType, nullability: Option<bool>) -> Result<SType> {
let kind = match ty {
ConcreteDataType::Null(_) => None,
ConcreteDataType::Boolean(_) => build_substrait_kind!(Bool, Boolean, nullability, 0),
ConcreteDataType::Int8(_) => build_substrait_kind!(I8, I8, nullability, 0),
ConcreteDataType::Int16(_) => build_substrait_kind!(I16, I16, nullability, 0),
ConcreteDataType::Int32(_) => build_substrait_kind!(I32, I32, nullability, 0),
ConcreteDataType::Int64(_) => build_substrait_kind!(I64, I64, nullability, 0),
ConcreteDataType::UInt8(_) => build_substrait_kind!(I8, I8, nullability, 1),
ConcreteDataType::UInt16(_) => build_substrait_kind!(I16, I16, nullability, 1),
ConcreteDataType::UInt32(_) => build_substrait_kind!(I32, I32, nullability, 1),
ConcreteDataType::UInt64(_) => build_substrait_kind!(I64, I64, nullability, 1),
ConcreteDataType::Float32(_) => build_substrait_kind!(Fp32, Fp32, nullability, 0),
ConcreteDataType::Float64(_) => build_substrait_kind!(Fp64, Fp64, nullability, 0),
ConcreteDataType::Binary(_) => build_substrait_kind!(Binary, Binary, nullability, 0),
ConcreteDataType::String(_) => build_substrait_kind!(String, String, nullability, 0),
ConcreteDataType::Date(_) => build_substrait_kind!(Date, Date, nullability, 0),
ConcreteDataType::DateTime(_) => UnsupportedConcreteTypeSnafu { ty }.fail()?,
ConcreteDataType::Timestamp(ty) => {
build_substrait_kind!(Timestamp, Timestamp, nullability, ty.precision() as u32)
}
ConcreteDataType::List(_) | ConcreteDataType::Dictionary(_) => {
UnsupportedConcreteTypeSnafu { ty }.fail()?
}
};
Ok(SType { kind })
}
pub(crate) fn scalar_value_as_literal_type(v: &ScalarValue) -> Result<LiteralType> {
Ok(if v.is_null() {
LiteralType::Null(Type { kind: None })
} else {
match v {
ScalarValue::Boolean(Some(v)) => LiteralType::Boolean(*v),
ScalarValue::Float32(Some(v)) => LiteralType::Fp32(*v),
ScalarValue::Float64(Some(v)) => LiteralType::Fp64(*v),
ScalarValue::Int8(Some(v)) => LiteralType::I8(*v as i32),
ScalarValue::Int16(Some(v)) => LiteralType::I16(*v as i32),
ScalarValue::Int32(Some(v)) => LiteralType::I32(*v),
ScalarValue::Int64(Some(v)) => LiteralType::I64(*v),
ScalarValue::LargeUtf8(Some(v)) => LiteralType::String(v.clone()),
ScalarValue::LargeBinary(Some(v)) => LiteralType::Binary(v.clone()),
ScalarValue::TimestampSecond(Some(seconds), _) => {
LiteralType::Timestamp(*seconds * 1_000_000)
}
ScalarValue::TimestampMillisecond(Some(millis), _) => {
LiteralType::Timestamp(*millis * 1000)
}
ScalarValue::TimestampMicrosecond(Some(micros), _) => LiteralType::Timestamp(*micros),
ScalarValue::TimestampNanosecond(Some(nanos), _) => {
LiteralType::Timestamp(*nanos / 1000)
}
ScalarValue::Utf8(Some(s)) => LiteralType::String(s.clone()),
// TODO(LFC): Implement other conversions: ScalarValue => LiteralType
_ => {
return error::UnsupportedExprSnafu {
name: format!("ScalarValue: {v:?}"),
}
.fail()
}
}
})
}
pub(crate) fn literal_type_to_scalar_value(t: LiteralType) -> Result<ScalarValue> {
Ok(match t {
LiteralType::Null(Type { kind: Some(kind) }) => match kind {
Kind::Bool(_) => ScalarValue::Boolean(None),
Kind::I8(_) => ScalarValue::Int8(None),
Kind::I16(_) => ScalarValue::Int16(None),
Kind::I32(_) => ScalarValue::Int32(None),
Kind::I64(_) => ScalarValue::Int64(None),
Kind::Fp32(_) => ScalarValue::Float32(None),
Kind::Fp64(_) => ScalarValue::Float64(None),
Kind::String(_) => ScalarValue::LargeUtf8(None),
Kind::Binary(_) => ScalarValue::LargeBinary(None),
// TODO(LFC): Implement other conversions: Kind => ScalarValue
_ => {
return error::UnsupportedSubstraitTypeSnafu {
ty: format!("{kind:?}"),
}
.fail()
}
},
LiteralType::Boolean(v) => ScalarValue::Boolean(Some(v)),
LiteralType::I8(v) => ScalarValue::Int8(Some(v as i8)),
LiteralType::I16(v) => ScalarValue::Int16(Some(v as i16)),
LiteralType::I32(v) => ScalarValue::Int32(Some(v)),
LiteralType::I64(v) => ScalarValue::Int64(Some(v)),
LiteralType::Fp32(v) => ScalarValue::Float32(Some(v)),
LiteralType::Fp64(v) => ScalarValue::Float64(Some(v)),
LiteralType::String(v) => ScalarValue::LargeUtf8(Some(v)),
LiteralType::Binary(v) => ScalarValue::LargeBinary(Some(v)),
LiteralType::Timestamp(v) => ScalarValue::TimestampMicrosecond(Some(v), None),
// TODO(LFC): Implement other conversions: LiteralType => ScalarValue
_ => {
return error::UnsupportedSubstraitTypeSnafu {
ty: format!("{t:?}"),
}
.fail()
}
})
}

View File

@@ -37,7 +37,6 @@ futures-util.workspace = true
key-lock = "0.1"
hyper = { version = "0.14", features = ["full"] }
humantime-serde = "1.1"
log = "0.4"
log-store = { path = "../log-store" }
meta-client = { path = "../meta-client" }
meta-srv = { path = "../meta-srv", features = ["mock"] }

View File

@@ -47,12 +47,20 @@ pub enum ObjectStoreConfig {
File(FileConfig),
S3(S3Config),
Oss(OssConfig),
Azblob(AzblobConfig),
}
/// Storage engine config
#[derive(Debug, Clone, Serialize, Deserialize, Default)]
#[serde(default)]
pub struct StorageConfig {
/// Retention period for all tables.
///
/// Default value is `None`, which means no TTL.
///
/// The precedence order is: ttl in table options > global ttl.
#[serde(with = "humantime_serde")]
pub global_ttl: Option<Duration>,
#[serde(flatten)]
pub store: ObjectStoreConfig,
pub compaction: CompactionConfig,
@@ -95,6 +103,21 @@ pub struct OssConfig {
pub cache_capacity: Option<ReadableSize>,
}
#[derive(Debug, Clone, Serialize, Deserialize)]
#[serde(default)]
pub struct AzblobConfig {
pub container: String,
pub root: String,
#[serde(skip_serializing)]
pub account_name: SecretString,
#[serde(skip_serializing)]
pub account_key: SecretString,
pub endpoint: String,
pub sas_token: Option<String>,
pub cache_path: Option<String>,
pub cache_capacity: Option<ReadableSize>,
}
impl Default for S3Config {
fn default() -> Self {
Self {
@@ -124,6 +147,21 @@ impl Default for OssConfig {
}
}
impl Default for AzblobConfig {
fn default() -> Self {
Self {
container: String::default(),
root: String::default(),
account_name: SecretString::from(String::default()),
account_key: SecretString::from(String::default()),
endpoint: String::default(),
cache_path: Option::default(),
cache_capacity: Option::default(),
sas_token: Option::default(),
}
}
}
impl Default for ObjectStoreConfig {
fn default() -> Self {
ObjectStoreConfig::File(FileConfig {
@@ -269,6 +307,7 @@ impl From<&DatanodeOptions> for StorageEngineConfig {
picker_schedule_interval: value.storage.flush.picker_schedule_interval,
auto_flush_interval: value.storage.flush.auto_flush_interval,
global_write_buffer_size: value.storage.flush.global_write_buffer_size,
global_ttl: value.storage.global_ttl,
}
}
}

View File

@@ -230,6 +230,12 @@ pub enum Error {
source: servers::error::Error,
},
#[snafu(display("Failed to wait for GRPC serving, source: {}", source))]
WaitForGrpcServing {
source: servers::error::Error,
location: Location,
},
#[snafu(display("Failed to parse address {}, source: {}", addr, source))]
ParseAddr {
addr: String,
@@ -467,6 +473,12 @@ pub enum Error {
#[snafu(display("Missing WAL dir config"))]
MissingWalDirConfig { location: Location },
#[snafu(display("Failed to join task, source: {}", source))]
JoinTask {
source: common_runtime::JoinError,
location: Location,
},
}
pub type Result<T> = std::result::Result<T, Error>;
@@ -537,16 +549,19 @@ impl ErrorExt for Error {
| GetTable { source, .. } => source.status_code(),
// TODO(yingwen): Further categorize http error.
StartServer { .. }
| ParseAddr { .. }
ParseAddr { .. }
| CreateDir { .. }
| RemoveDir { .. }
| Catalog { .. }
| MissingRequiredField { .. }
| IncorrectInternalState { .. }
| ShutdownServer { .. }
| ShutdownInstance { .. }
| CloseTableEngine { .. } => StatusCode::Internal,
| CloseTableEngine { .. }
| JoinTask { .. } => StatusCode::Internal,
StartServer { source }
| ShutdownServer { source }
| WaitForGrpcServing { source, .. } => source.status_code(),
InitBackend { .. } => StatusCode::StorageUnavailable,

View File

@@ -20,7 +20,9 @@ use common_meta::error::Result as MetaResult;
use common_meta::heartbeat::handler::{
HandleControl, HeartbeatResponseHandler, HeartbeatResponseHandlerContext,
};
use common_meta::instruction::{Instruction, InstructionReply, RegionIdent, SimpleReply};
use common_meta::instruction::{
Instruction, InstructionReply, RegionIdent, SimpleReply, TableIdent,
};
use common_telemetry::{error, info, warn};
use snafu::ResultExt;
use store_api::storage::RegionNumber;
@@ -53,10 +55,14 @@ impl HeartbeatResponseHandler for CloseRegionHandler {
let self_ref = Arc::new(self.clone());
let RegionIdent {
engine,
catalog,
schema,
table,
table_ident:
TableIdent {
engine,
catalog,
schema,
table,
..
},
region_number,
..
} = region_ident;
@@ -186,6 +192,7 @@ impl CloseRegionHandler {
schema_name: table_ref.schema.to_string(),
table_name: table_ref.table.to_string(),
region_numbers: region_numbers.clone(),
flush: true,
},
)
.await

View File

@@ -21,7 +21,9 @@ use common_meta::error::Result as MetaResult;
use common_meta::heartbeat::handler::{
HandleControl, HeartbeatResponseHandler, HeartbeatResponseHandlerContext,
};
use common_meta::instruction::{Instruction, InstructionReply, RegionIdent, SimpleReply};
use common_meta::instruction::{
Instruction, InstructionReply, RegionIdent, SimpleReply, TableIdent,
};
use common_telemetry::{error, warn};
use snafu::ResultExt;
use store_api::storage::RegionNumber;
@@ -97,12 +99,15 @@ impl OpenRegionHandler {
fn prepare_request(ident: RegionIdent) -> (String, OpenTableRequest) {
let RegionIdent {
catalog,
schema,
table,
table_id,
table_ident:
TableIdent {
catalog,
schema,
table,
table_id,
engine,
},
region_number,
engine,
..
} = ident;

View File

@@ -18,20 +18,22 @@ use std::sync::Arc;
use api::v1::ddl_request::Expr as DdlExpr;
use api::v1::greptime_request::Request as GrpcRequest;
use api::v1::query_request::Query;
use api::v1::{CreateDatabaseExpr, DdlRequest, DeleteRequest, InsertRequest};
use api::v1::{CreateDatabaseExpr, DdlRequest, DeleteRequest, InsertRequests};
use async_trait::async_trait;
use catalog::CatalogManagerRef;
use common_grpc_expr::insert::to_table_insert_request;
use common_query::Output;
use datafusion::catalog::catalog::{
CatalogList, CatalogProvider, MemoryCatalogList, MemoryCatalogProvider,
};
use datafusion::catalog::schema::SchemaProvider;
use datafusion::datasource::TableProvider;
use futures::future;
use query::parser::{PromQuery, QueryLanguageParser, QueryStatement};
use query::plan::LogicalPlan;
use query::query_engine::SqlStatementExecutor;
use servers::query_handler::grpc::GrpcQueryHandler;
use session::context::{QueryContext, QueryContextRef};
use session::context::QueryContextRef;
use snafu::prelude::*;
use sql::statements::statement::Statement;
use substrait::{DFLogicalSubstraitConvertor, SubstraitPlan};
@@ -41,8 +43,8 @@ use table::table::adapter::DfTableProviderAdapter;
use crate::error::{
self, CatalogNotFoundSnafu, CatalogSnafu, DecodeLogicalPlanSnafu, DeleteExprToRequestSnafu,
DeleteSnafu, ExecuteLogicalPlanSnafu, ExecuteSqlSnafu, InsertSnafu, PlanStatementSnafu, Result,
SchemaNotFoundSnafu, TableNotFoundSnafu,
DeleteSnafu, ExecuteLogicalPlanSnafu, ExecuteSqlSnafu, InsertDataSnafu, InsertSnafu,
JoinTaskSnafu, PlanStatementSnafu, Result, SchemaNotFoundSnafu, TableNotFoundSnafu,
};
use crate::instance::Instance;
@@ -77,7 +79,7 @@ impl Instance {
.context(DecodeLogicalPlanSnafu)?;
self.query_engine
.execute(LogicalPlan::DfPlan(logical_plan), QueryContext::arc())
.execute(LogicalPlan::DfPlan(logical_plan), ctx.clone())
.await
.context(ExecuteLogicalPlanSnafu)
}
@@ -119,31 +121,41 @@ impl Instance {
}
}
pub async fn handle_insert(
pub async fn handle_inserts(
&self,
request: InsertRequest,
ctx: QueryContextRef,
requests: InsertRequests,
ctx: &QueryContextRef,
) -> Result<Output> {
let catalog = &ctx.current_catalog();
let schema = &ctx.current_schema();
let table_name = &request.table_name.clone();
let table_ref = TableReference::full(catalog, schema, table_name);
let results = future::try_join_all(requests.inserts.into_iter().map(|insert| {
let catalog_manager = self.catalog_manager.clone();
let catalog = ctx.current_catalog();
let schema = ctx.current_schema();
let table = self
.catalog_manager
.table(catalog, schema, table_name)
.await
.context(CatalogSnafu)?
.with_context(|| TableNotFoundSnafu {
table_name: table_ref.to_string(),
})?;
common_runtime::spawn_write(async move {
let table_name = &insert.table_name.clone();
let table = catalog_manager
.table(&catalog, &schema, table_name)
.await
.context(CatalogSnafu)?
.with_context(|| TableNotFoundSnafu {
table_name: common_catalog::format_full_table_name(
&catalog, &schema, table_name,
),
})?;
let request = common_grpc_expr::insert::to_table_insert_request(catalog, schema, request)
.context(error::InsertDataSnafu)?;
let request =
to_table_insert_request(&catalog, &schema, insert).context(InsertDataSnafu)?;
let affected_rows = table.insert(request).await.with_context(|_| InsertSnafu {
table_name: table_ref.to_string(),
})?;
table.insert(request).await.with_context(|_| InsertSnafu {
table_name: common_catalog::format_full_table_name(
&catalog, &schema, table_name,
),
})
})
}))
.await
.context(JoinTaskSnafu)?;
let affected_rows = results.into_iter().sum::<Result<usize>>()?;
Ok(Output::AffectedRows(affected_rows))
}
@@ -191,7 +203,7 @@ impl GrpcQueryHandler for Instance {
async fn do_query(&self, request: GrpcRequest, ctx: QueryContextRef) -> Result<Output> {
match request {
GrpcRequest::Insert(request) => self.handle_insert(request, ctx).await,
GrpcRequest::Inserts(requests) => self.handle_inserts(requests, &ctx).await,
GrpcRequest::Delete(request) => self.handle_delete(request, ctx).await,
GrpcRequest::Query(query_request) => {
let query = query_request
@@ -291,10 +303,12 @@ async fn new_dummy_catalog_list(
#[cfg(test)]
mod test {
use api::v1::add_column::location::LocationType;
use api::v1::add_column::Location;
use api::v1::column::{SemanticType, Values};
use api::v1::{
alter_expr, AddColumn, AddColumns, AlterExpr, Column, ColumnDataType, ColumnDef,
CreateDatabaseExpr, CreateTableExpr, QueryRequest,
CreateDatabaseExpr, CreateTableExpr, InsertRequest, InsertRequests, QueryRequest,
};
use common_catalog::consts::MITO_ENGINE;
use common_recordbatch::RecordBatches;
@@ -364,15 +378,44 @@ mod test {
schema_name: "my_database".to_string(),
table_name: "my_table".to_string(),
kind: Some(alter_expr::Kind::AddColumns(AddColumns {
add_columns: vec![AddColumn {
column_def: Some(ColumnDef {
name: "b".to_string(),
datatype: ColumnDataType::Int32 as i32,
is_nullable: true,
default_constraint: vec![],
}),
is_key: true,
}],
add_columns: vec![
AddColumn {
column_def: Some(ColumnDef {
name: "b".to_string(),
datatype: ColumnDataType::Int32 as i32,
is_nullable: true,
default_constraint: vec![],
}),
is_key: true,
location: None,
},
AddColumn {
column_def: Some(ColumnDef {
name: "c".to_string(),
datatype: ColumnDataType::Int32 as i32,
is_nullable: true,
default_constraint: vec![],
}),
is_key: true,
location: Some(Location {
location_type: LocationType::First.into(),
after_cloumn_name: "".to_string(),
}),
},
AddColumn {
column_def: Some(ColumnDef {
name: "d".to_string(),
datatype: ColumnDataType::Int32 as i32,
is_nullable: true,
default_constraint: vec![],
}),
is_key: true,
location: Some(Location {
location_type: LocationType::After.into(),
after_cloumn_name: "a".to_string(),
}),
},
],
})),
})),
});
@@ -388,15 +431,15 @@ mod test {
.unwrap();
assert!(matches!(output, Output::AffectedRows(1)));
let output = exec_selection(instance, "SELECT ts, a, b FROM my_database.my_table").await;
let output = exec_selection(instance, "SELECT * FROM my_database.my_table").await;
let Output::Stream(stream) = output else { unreachable!() };
let recordbatches = RecordBatches::try_collect(stream).await.unwrap();
let expected = "\
+---------------------+---+---+
| ts | a | b |
+---------------------+---+---+
| 2022-12-30T07:09:00 | s | 1 |
+---------------------+---+---+";
+---+---+---+---------------------+---+
| c | a | d | ts | b |
+---+---+---+---------------------+---+
| | s | | 2022-12-30T07:09:00 | 1 |
+---+---+---+---------------------+---+";
assert_eq!(recordbatches.pretty_print().unwrap(), expected);
}
@@ -450,7 +493,9 @@ mod test {
..Default::default()
};
let query = GrpcRequest::Insert(insert);
let query = GrpcRequest::Inserts(InsertRequests {
inserts: vec![insert],
});
let output = instance.do_query(query, QueryContext::arc()).await.unwrap();
assert!(matches!(output, Output::AffectedRows(3)));

View File

@@ -17,17 +17,18 @@ use std::net::SocketAddr;
use std::sync::Arc;
use common_runtime::Builder as RuntimeBuilder;
use futures::future;
use servers::grpc::GrpcServer;
use servers::http::{HttpServer, HttpServerBuilder};
use servers::metrics_handler::MetricsHandler;
use servers::query_handler::grpc::ServerGrpcQueryHandlerAdaptor;
use servers::server::Server;
use snafu::ResultExt;
use tokio::select;
use crate::datanode::DatanodeOptions;
use crate::error::{
ParseAddrSnafu, Result, RuntimeResourceSnafu, ShutdownServerSnafu, StartServerSnafu,
WaitForGrpcServingSnafu,
};
use crate::instance::InstanceRef;
@@ -71,10 +72,14 @@ impl Services {
})?;
let grpc = self.grpc_server.start(grpc_addr);
let http = self.http_server.start(http_addr);
select!(
v = grpc => v.context(StartServerSnafu)?,
v = http => v.context(StartServerSnafu)?,
);
future::try_join_all(vec![grpc, http])
.await
.context(StartServerSnafu)?;
self.grpc_server
.wait_for_serve()
.await
.context(WaitForGrpcServingSnafu)?;
Ok(())
}

View File

@@ -97,7 +97,7 @@ impl Instance {
catalog_name: expr.catalog_name,
schema_name: expr.schema_name,
table_name,
region_number: expr.region_id,
region_number: expr.region_number,
wait: None,
};
self.sql_handler()
@@ -239,7 +239,7 @@ mod tests {
table_id: Some(TableId {
id: MIN_USER_TABLE_ID,
}),
region_ids: vec![0],
region_numbers: vec![0],
engine: MITO_ENGINE.to_string(),
}
}

View File

@@ -68,12 +68,16 @@ impl SqlHandler {
}
.fail()
}
AlterTableOperation::AddColumn { column_def } => AlterKind::AddColumns {
AlterTableOperation::AddColumn {
column_def,
location,
} => AlterKind::AddColumns {
columns: vec![AddColumnRequest {
column_schema: column_def_to_schema(column_def, false)
.context(error::ParseSqlSnafu)?,
// FIXME(dennis): supports adding key column
is_key: false,
location: location.clone(),
}],
},
AlterTableOperation::DropColumn { name } => AlterKind::DropColumns {

View File

@@ -14,6 +14,7 @@
//! object storage utilities
mod azblob;
mod fs;
mod oss;
mod s3;
@@ -25,7 +26,7 @@ use common_base::readable_size::ReadableSize;
use common_telemetry::logging::info;
use object_store::layers::{LoggingLayer, LruCacheLayer, MetricsLayer, RetryLayer, TracingLayer};
use object_store::services::Fs as FsBuilder;
use object_store::{ObjectStore, ObjectStoreBuilder};
use object_store::{util, ObjectStore, ObjectStoreBuilder};
use snafu::prelude::*;
use crate::datanode::{ObjectStoreConfig, DEFAULT_OBJECT_STORE_CACHE_SIZE};
@@ -36,6 +37,9 @@ pub(crate) async fn new_object_store(store_config: &ObjectStoreConfig) -> Result
ObjectStoreConfig::File(file_config) => fs::new_fs_object_store(file_config).await,
ObjectStoreConfig::S3(s3_config) => s3::new_s3_object_store(s3_config).await,
ObjectStoreConfig::Oss(oss_config) => oss::new_oss_object_store(oss_config).await,
ObjectStoreConfig::Azblob(azblob_config) => {
azblob::new_azblob_object_store(azblob_config).await
}
}?;
// Enable retry layer and cache layer for non-fs object storages
@@ -52,7 +56,8 @@ pub(crate) async fn new_object_store(store_config: &ObjectStoreConfig) -> Result
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(log::Level::Debug)),
.with_error_level(Some("debug"))
.expect("input error level must be valid"),
)
.layer(TracingLayer))
}
@@ -76,14 +81,22 @@ async fn create_object_store_with_cache(
.unwrap_or(DEFAULT_OBJECT_STORE_CACHE_SIZE);
(path, capacity)
}
ObjectStoreConfig::Azblob(azblob_config) => {
let path = azblob_config.cache_path.as_ref();
let capacity = azblob_config
.cache_capacity
.unwrap_or(DEFAULT_OBJECT_STORE_CACHE_SIZE);
(path, capacity)
}
_ => (None, ReadableSize(0)),
};
if let Some(path) = cache_path {
let atomic_temp_dir = format!("{path}/.tmp/");
let path = util::normalize_dir(path);
let atomic_temp_dir = format!("{path}.tmp/");
clean_temp_dir(&atomic_temp_dir)?;
let cache_store = FsBuilder::default()
.root(path)
.root(&path)
.atomic_write_dir(&atomic_temp_dir)
.build()
.context(error::InitBackendSnafu)?;

View File

@@ -0,0 +1,47 @@
// 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::logging::info;
use object_store::services::Azblob as AzureBuilder;
use object_store::{util, ObjectStore};
use secrecy::ExposeSecret;
use snafu::prelude::*;
use crate::datanode::AzblobConfig;
use crate::error::{self, Result};
pub(crate) async fn new_azblob_object_store(azblob_config: &AzblobConfig) -> Result<ObjectStore> {
let root = util::normalize_dir(&azblob_config.root);
info!(
"The azure storage container is: {}, root is: {}",
azblob_config.container, &root
);
let mut builder = AzureBuilder::default();
builder
.root(&root)
.container(&azblob_config.container)
.endpoint(&azblob_config.endpoint)
.account_name(azblob_config.account_name.expose_secret())
.account_key(azblob_config.account_key.expose_secret());
if let Some(token) = &azblob_config.sas_token {
builder.sas_token(token);
}
Ok(ObjectStore::new(builder)
.context(error::InitBackendSnafu)?
.finish())
}

View File

@@ -29,7 +29,7 @@ pub(crate) async fn new_fs_object_store(file_config: &FileConfig) -> Result<Obje
.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 = format!("{data_home}.tmp/");
store::clean_temp_dir(&atomic_write_dir)?;
let mut builder = FsBuilder::default();

View File

@@ -24,7 +24,9 @@ use common_meta::heartbeat::handler::{
HandlerGroupExecutor, HeartbeatResponseHandlerContext, HeartbeatResponseHandlerExecutor,
};
use common_meta::heartbeat::mailbox::{HeartbeatMailbox, MessageMeta};
use common_meta::instruction::{Instruction, InstructionReply, RegionIdent, SimpleReply};
use common_meta::instruction::{
Instruction, InstructionReply, RegionIdent, SimpleReply, TableIdent,
};
use common_query::Output;
use datatypes::prelude::ConcreteDataType;
use servers::query_handler::grpc::GrpcQueryHandler;
@@ -95,11 +97,13 @@ async fn test_close_region_handler() {
executor.clone(),
mailbox.clone(),
Instruction::CloseRegion(RegionIdent {
catalog: "greptime".to_string(),
schema: "public".to_string(),
table: "non-exist".to_string(),
table_id: 1025,
engine: "mito".to_string(),
table_ident: TableIdent {
catalog: "greptime".to_string(),
schema: "public".to_string(),
table: "non-exist".to_string(),
table_id: 1025,
engine: "mito".to_string(),
},
region_number: 0,
cluster_id: 1,
datanode_id: 2,
@@ -149,11 +153,13 @@ async fn test_open_region_handler() {
executor.clone(),
mailbox.clone(),
Instruction::OpenRegion(RegionIdent {
catalog: "greptime".to_string(),
schema: "public".to_string(),
table: "non-exist".to_string(),
table_id: 2024,
engine: "mito".to_string(),
table_ident: TableIdent {
catalog: "greptime".to_string(),
schema: "public".to_string(),
table: "non-exist".to_string(),
table_id: 2024,
engine: "mito".to_string(),
},
region_number: 0,
cluster_id: 1,
datanode_id: 2,
@@ -228,11 +234,13 @@ fn handle_instruction(
fn close_region_instruction() -> Instruction {
Instruction::CloseRegion(RegionIdent {
catalog: "greptime".to_string(),
schema: "public".to_string(),
table: "demo".to_string(),
table_id: 1024,
engine: "mito".to_string(),
table_ident: TableIdent {
catalog: "greptime".to_string(),
schema: "public".to_string(),
table: "demo".to_string(),
table_id: 1024,
engine: "mito".to_string(),
},
region_number: 0,
cluster_id: 1,
datanode_id: 2,
@@ -241,11 +249,13 @@ fn close_region_instruction() -> Instruction {
fn open_region_instruction() -> Instruction {
Instruction::OpenRegion(RegionIdent {
catalog: "greptime".to_string(),
schema: "public".to_string(),
table: "demo".to_string(),
table_id: 1024,
engine: "mito".to_string(),
table_ident: TableIdent {
catalog: "greptime".to_string(),
schema: "public".to_string(),
table: "demo".to_string(),
table_id: 1024,
engine: "mito".to_string(),
},
region_number: 0,
cluster_id: 1,
datanode_id: 2,

View File

@@ -73,13 +73,6 @@ pub enum Error {
#[snafu(display("Invalid timestamp index: {}", index))]
InvalidTimestampIndex { index: usize, location: Location },
#[snafu(display("Duplicate timestamp index, exists: {}, new: {}", exists, new))]
DuplicateTimestampIndex {
exists: usize,
new: usize,
location: Location,
},
#[snafu(display("{}", msg))]
CastType { msg: String, location: Location },

View File

@@ -237,14 +237,7 @@ fn collect_fields(column_schemas: &[ColumnSchema]) -> Result<FieldsAndIndices> {
let mut name_to_index = HashMap::with_capacity(column_schemas.len());
let mut timestamp_index = None;
for (index, column_schema) in column_schemas.iter().enumerate() {
if column_schema.is_time_index() {
ensure!(
timestamp_index.is_none(),
error::DuplicateTimestampIndexSnafu {
exists: timestamp_index.unwrap(),
new: index,
}
);
if column_schema.is_time_index() && timestamp_index.is_none() {
timestamp_index = Some(index);
}
let field = Field::try_from(column_schema)?;
@@ -302,14 +295,8 @@ impl TryFrom<Arc<ArrowSchema>> for Schema {
for (index, column_schema) in column_schemas.iter().enumerate() {
if column_schema.is_time_index() {
validate_timestamp_index(&column_schemas, index)?;
ensure!(
timestamp_index.is_none(),
error::DuplicateTimestampIndexSnafu {
exists: timestamp_index.unwrap(),
new: index,
}
);
timestamp_index = Some(index);
break;
}
}

View File

@@ -113,6 +113,14 @@ impl Vector for BinaryVector {
}
}
impl From<Vec<Vec<u8>>> for BinaryVector {
fn from(data: Vec<Vec<u8>>) -> Self {
Self {
array: BinaryArray::from_iter_values(data),
}
}
}
impl ScalarVector for BinaryVector {
type OwnedItem = Vec<u8>;
type RefItem<'a> = &'a [u8];

View File

@@ -433,8 +433,7 @@ impl NullBufferBuilder {
/// Builds the null buffer and resets the builder.
/// Returns `None` if the builder only contains `true`s.
fn finish(&mut self) -> Option<Buffer> {
let buf = self.bitmap_builder.as_mut().map(|b| b.finish());
self.bitmap_builder = None;
let buf = self.bitmap_builder.take().map(Into::into);
self.len = 0;
buf
}

View File

@@ -130,6 +130,12 @@ impl<T: LogicalPrimitiveType> PrimitiveVector<T> {
}
}
pub fn from_iter_values<I: IntoIterator<Item = T::Native>>(iter: I) -> Self {
Self {
array: PrimitiveArray::from_iter_values(iter),
}
}
pub fn from_values<I: IntoIterator<Item = T::Native>>(iter: I) -> Self {
Self {
array: PrimitiveArray::from_iter_values(iter),

View File

@@ -16,5 +16,5 @@ pub mod immutable;
#[inline]
pub fn table_manifest_dir(table_dir: &str) -> String {
format!("{table_dir}/manifest/")
format!("{table_dir}manifest/")
}

View File

@@ -108,7 +108,7 @@ mod tests {
write_table_manifest(
TEST_TABLE_NAME,
&table_manifest_dir(TEST_TABLE_NAME),
&table_manifest_dir(&format!("{TEST_TABLE_NAME}/")),
&store,
&metadata,
)
@@ -118,7 +118,7 @@ mod tests {
// try to overwrite immutable manifest
let write_immutable = write_table_manifest(
TEST_TABLE_NAME,
&table_manifest_dir(TEST_TABLE_NAME),
&table_manifest_dir(&format!("{TEST_TABLE_NAME}/")),
&store,
&metadata,
)
@@ -135,7 +135,7 @@ mod tests {
write_table_manifest(
TEST_TABLE_NAME,
&table_manifest_dir(TEST_TABLE_NAME),
&table_manifest_dir(&format!("{TEST_TABLE_NAME}/")),
&store,
&metadata,
)
@@ -144,7 +144,7 @@ mod tests {
let read = read_table_manifest(
TEST_TABLE_NAME,
&table_manifest_dir(TEST_TABLE_NAME),
&table_manifest_dir(&format!("{TEST_TABLE_NAME}/")),
&store,
)
.await
@@ -158,7 +158,7 @@ mod tests {
let (_dir, store) = new_test_object_store("test_read_non_exist_table_manifest");
let not_fount = read_table_manifest(
TEST_TABLE_NAME,
&table_manifest_dir(TEST_TABLE_NAME),
&table_manifest_dir(&format!("{TEST_TABLE_NAME}/")),
&store,
)
.await
@@ -172,7 +172,7 @@ mod tests {
let (_dir, store) = new_test_object_store("test_delete_table_manifest");
let metadata = build_test_table_metadata();
let table_dir = &table_manifest_dir(TEST_TABLE_NAME);
let table_dir = &table_manifest_dir(&format!("{TEST_TABLE_NAME}/"));
write_table_manifest(TEST_TABLE_NAME, table_dir, &store, &metadata)
.await
.unwrap();

View File

@@ -37,7 +37,7 @@ datatypes = { path = "../datatypes" }
file-table-engine = { path = "../file-table-engine" }
futures = "0.3"
futures-util.workspace = true
itertools = "0.10"
itertools.workspace = true
meta-client = { path = "../meta-client" }
meter-core.workspace = true
meter-macros.workspace = true
@@ -65,6 +65,7 @@ tokio.workspace = true
tonic.workspace = true
[dev-dependencies]
catalog = { path = "../catalog", features = ["testing"] }
common-test-util = { path = "../common/test-util" }
datanode = { path = "../datanode" }
futures = "0.3"

View File

@@ -32,6 +32,7 @@ use catalog::{
RegisterSchemaRequest, RegisterSystemTableRequest, RegisterTableRequest, RenameTableRequest,
SchemaProvider, SchemaProviderRef,
};
use client::client_manager::DatanodeClients;
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
use common_error::prelude::BoxedError;
use common_meta::table_name::TableName;
@@ -43,7 +44,6 @@ use snafu::prelude::*;
use table::table::numbers::NumbersTable;
use table::TableRef;
use crate::datanode::DatanodeClients;
use crate::expr_factory;
use crate::instance::distributed::DistInstance;
use crate::table::DistTable;
@@ -51,7 +51,7 @@ use crate::table::DistTable;
#[derive(Clone)]
pub struct FrontendCatalogManager {
backend: KvBackendRef,
backend_cache_invalidtor: KvCacheInvalidatorRef,
backend_cache_invalidator: KvCacheInvalidatorRef,
partition_manager: PartitionRuleManagerRef,
datanode_clients: Arc<DatanodeClients>,
@@ -65,13 +65,13 @@ pub struct FrontendCatalogManager {
impl FrontendCatalogManager {
pub fn new(
backend: KvBackendRef,
backend_cache_invalidtor: KvCacheInvalidatorRef,
backend_cache_invalidator: KvCacheInvalidatorRef,
partition_manager: PartitionRuleManagerRef,
datanode_clients: Arc<DatanodeClients>,
) -> Self {
Self {
backend,
backend_cache_invalidtor,
backend_cache_invalidator,
partition_manager,
datanode_clients,
dist_instance: None,
@@ -103,7 +103,7 @@ impl FrontendCatalogManager {
let key = schema_key.as_bytes();
self.backend_cache_invalidtor.invalidate_key(key).await;
self.backend_cache_invalidator.invalidate_key(key).await;
}
pub async fn invalidate_table(&self, catalog: &str, schema: &str, table: &str) {
@@ -116,7 +116,7 @@ impl FrontendCatalogManager {
let tg_key = tg_key.as_bytes();
self.backend_cache_invalidtor.invalidate_key(tg_key).await;
self.backend_cache_invalidator.invalidate_key(tg_key).await;
}
}
@@ -232,7 +232,7 @@ impl CatalogManager for FrontendCatalogManager {
create_if_not_exists: request.create_if_not_exists,
table_options: (&request.table_options).into(),
table_id: None, // Should and will be assigned by Meta.
region_ids: vec![0],
region_numbers: vec![0],
engine: request.engine,
};
@@ -279,9 +279,7 @@ impl CatalogManager for FrontendCatalogManager {
Ok(self.backend.get(key.as_bytes()).await?.map(|_| {
Arc::new(FrontendCatalogProvider {
catalog_name: catalog.to_string(),
backend: self.backend.clone(),
partition_manager: self.partition_manager.clone(),
datanode_clients: self.datanode_clients.clone(),
catalog_manager: Arc::new(self.clone()),
}) as Arc<_>
}))
}
@@ -320,9 +318,7 @@ impl CatalogManager for FrontendCatalogManager {
pub struct FrontendCatalogProvider {
catalog_name: String,
backend: KvBackendRef,
partition_manager: PartitionRuleManagerRef,
datanode_clients: Arc<DatanodeClients>,
catalog_manager: Arc<FrontendCatalogManager>,
}
#[async_trait::async_trait]
@@ -333,7 +329,8 @@ impl CatalogProvider for FrontendCatalogProvider {
async fn schema_names(&self) -> catalog::error::Result<Vec<String>> {
let key = build_schema_prefix(&self.catalog_name);
let mut iter = self.backend.range(key.as_bytes());
let backend = self.catalog_manager.backend();
let mut iter = backend.range(key.as_bytes());
let mut res = HashSet::new();
while let Some(r) = iter.next().await {
let Kv(k, _) = r?;
@@ -361,15 +358,17 @@ impl CatalogProvider for FrontendCatalogProvider {
}
.to_string();
let val = self.backend.get(schema_key.as_bytes()).await?;
let val = self
.catalog_manager
.backend()
.get(schema_key.as_bytes())
.await?;
let provider = val.map(|_| {
Arc::new(FrontendSchemaProvider {
catalog_name: catalog.clone(),
schema_name: name.to_string(),
backend: self.backend.clone(),
partition_manager: self.partition_manager.clone(),
datanode_clients: self.datanode_clients.clone(),
catalog_manager: self.catalog_manager.clone(),
}) as Arc<dyn SchemaProvider>
});
@@ -380,9 +379,7 @@ impl CatalogProvider for FrontendCatalogProvider {
pub struct FrontendSchemaProvider {
catalog_name: String,
schema_name: String,
backend: KvBackendRef,
partition_manager: PartitionRuleManagerRef,
datanode_clients: Arc<DatanodeClients>,
catalog_manager: Arc<FrontendCatalogManager>,
}
#[async_trait]
@@ -397,7 +394,8 @@ impl SchemaProvider for FrontendSchemaProvider {
tables.push("numbers".to_string());
}
let key = build_table_global_prefix(&self.catalog_name, &self.schema_name);
let iter = self.backend.range(key.as_bytes());
let backend = self.catalog_manager.backend();
let iter = backend.range(key.as_bytes());
let result = iter
.map(|r| {
let Kv(k, _) = r?;
@@ -424,7 +422,9 @@ impl SchemaProvider for FrontendSchemaProvider {
schema_name: self.schema_name.clone(),
table_name: name.to_string(),
};
let Some(kv) = self.backend.get(table_global_key.to_string().as_bytes()).await? else { return Ok(None) };
let Some(kv) = self.catalog_manager.backend().get(table_global_key.to_string().as_bytes()).await? else {
return Ok(None);
};
let v = TableGlobalValue::from_bytes(kv.1).context(InvalidCatalogValueSnafu)?;
let table_info = Arc::new(
v.table_info
@@ -434,9 +434,7 @@ impl SchemaProvider for FrontendSchemaProvider {
let table = Arc::new(DistTable::new(
TableName::new(&self.catalog_name, &self.schema_name, name),
table_info,
self.partition_manager.clone(),
self.datanode_clients.clone(),
self.backend.clone(),
self.catalog_manager.clone(),
));
Ok(Some(table))
}

View File

@@ -19,7 +19,7 @@ use datafusion::parquet;
use datatypes::arrow::error::ArrowError;
use datatypes::value::Value;
use snafu::Location;
use store_api::storage::RegionId;
use store_api::storage::RegionNumber;
#[derive(Debug, Snafu)]
#[snafu(visibility(pub))]
@@ -96,6 +96,9 @@ pub enum Error {
source: api::error::Error,
},
#[snafu(display("Failed to convert vector to GRPC column, reason: {}", reason))]
VectorToGrpcColumn { reason: String, location: Location },
#[snafu(display(
"Failed to convert column default constraint, column: {}, source: {}",
column_name,
@@ -115,7 +118,7 @@ pub enum Error {
#[snafu(display("Failed to find Datanode by region: {:?}", region))]
FindDatanode {
region: RegionId,
region: RegionNumber,
location: Location,
},
@@ -192,6 +195,12 @@ pub enum Error {
source: partition::error::Error,
},
#[snafu(display("Failed to split insert request, source: {}", source))]
SplitInsert {
source: partition::error::Error,
location: Location,
},
#[snafu(display("Failed to create table info, source: {}", source))]
CreateTableInfo {
#[snafu(backtrace)]
@@ -582,7 +591,8 @@ impl ErrorExt for Error {
| Error::CreateTableRoute { .. }
| Error::FindRegionRoute { .. }
| Error::BuildDfLogicalPlan { .. }
| Error::BuildTableMeta { .. } => StatusCode::Internal,
| Error::BuildTableMeta { .. }
| Error::VectorToGrpcColumn { .. } => StatusCode::Internal,
Error::IncompleteGrpcResult { .. }
| Error::ContextValueNotFound { .. }
@@ -623,7 +633,9 @@ impl ErrorExt for Error {
Error::External { source } => source.status_code(),
Error::DeserializePartition { source, .. }
| Error::FindTablePartitionRule { source, .. }
| Error::FindTableRoute { source, .. } => source.status_code(),
| Error::FindTableRoute { source, .. }
| Error::SplitInsert { source, .. } => source.status_code(),
Error::UnrecognizedTableOption { .. } => StatusCode::InvalidArguments,
Error::StartScriptManager { source } => source.status_code(),

View File

@@ -115,7 +115,7 @@ pub(crate) async fn create_external_expr(
create_if_not_exists: create.if_not_exists,
table_options: options,
table_id: None,
region_ids: vec![],
region_numbers: vec![],
engine: create.engine.to_string(),
};
Ok(expr)
@@ -144,7 +144,7 @@ pub fn create_to_expr(create: &CreateTable, query_ctx: QueryContextRef) -> Resul
create_if_not_exists: create.if_not_exists,
table_options,
table_id: None,
region_ids: vec![],
region_numbers: vec![],
engine: create.engine.to_string(),
};
Ok(expr)
@@ -291,7 +291,10 @@ pub(crate) fn to_alter_expr(
}
.fail();
}
AlterTableOperation::AddColumn { column_def } => Kind::AddColumns(AddColumns {
AlterTableOperation::AddColumn {
column_def,
location,
} => Kind::AddColumns(AddColumns {
add_columns: vec![AddColumn {
column_def: Some(
sql_column_def_to_grpc_column_def(column_def)
@@ -299,6 +302,7 @@ pub(crate) fn to_alter_expr(
.context(ExternalSnafu)?,
),
is_key: false,
location: location.as_ref().map(From::from),
}],
}),
AlterTableOperation::DropColumn { name } => Kind::DropColumns(DropColumns {

View File

@@ -13,5 +13,6 @@
// limitations under the License.
pub mod invalidate_table_cache;
#[cfg(test)]
mod tests;
pub(crate) mod tests;

View File

@@ -30,6 +30,7 @@ use tokio::sync::mpsc;
use super::invalidate_table_cache::InvalidateTableCacheHandler;
#[derive(Default)]
pub struct MockKvCacheInvalidator {
inner: Mutex<HashMap<Vec<u8>, i32>>,
}
@@ -82,8 +83,6 @@ async fn test_invalidate_table_cache_handler() {
executor.clone(),
mailbox.clone(),
Instruction::InvalidateTableCache(TableIdent {
cluster_id: 1,
datanode_id: 2,
catalog: "test".to_string(),
schema: "greptime".to_string(),
table: "foo_table".to_string(),
@@ -120,8 +119,6 @@ async fn test_invalidate_table_cache_handler() {
executor,
mailbox,
Instruction::InvalidateTableCache(TableIdent {
cluster_id: 1,
datanode_id: 2,
catalog: "test".to_string(),
schema: "greptime".to_string(),
table: "not_found".to_string(),

View File

@@ -28,10 +28,11 @@ use api::v1::alter_expr::Kind;
use api::v1::ddl_request::Expr as DdlExpr;
use api::v1::greptime_request::Request;
use api::v1::meta::Role;
use api::v1::{AddColumns, AlterExpr, Column, DdlRequest, InsertRequest};
use api::v1::{AddColumns, AlterExpr, Column, DdlRequest, InsertRequest, InsertRequests};
use async_trait::async_trait;
use catalog::remote::CachedMetaKvBackend;
use catalog::CatalogManagerRef;
use client::client_manager::DatanodeClients;
use common_base::Plugins;
use common_catalog::consts::MITO_ENGINE;
use common_error::ext::BoxedError;
@@ -46,6 +47,7 @@ use datanode::instance::sql::table_idents_to_full_name;
use datanode::instance::InstanceRef as DnInstanceRef;
use datatypes::schema::Schema;
use distributed::DistInstance;
use futures::future;
use meta_client::client::{MetaClient, MetaClientBuilder};
use meta_client::MetaClientOptions;
use partition::manager::PartitionRuleManager;
@@ -70,7 +72,6 @@ use sql::statements::copy::CopyTable;
use sql::statements::statement::Statement;
use crate::catalog::FrontendCatalogManager;
use crate::datanode::DatanodeClients;
use crate::error::{
self, Error, ExecutePromqlSnafu, ExternalSnafu, InvalidInsertRequestSnafu,
MissingMetasrvOptsSnafu, ParseSqlSnafu, PlanStatementSnafu, Result, SqlExecInterceptedSnafu,
@@ -98,7 +99,7 @@ pub trait FrontendInstance:
+ Sync
+ 'static
{
async fn start(&mut self) -> Result<()>;
async fn start(&self) -> Result<()>;
}
pub type FrontendInstanceRef = Arc<dyn FrontendInstance>;
@@ -154,16 +155,21 @@ impl Instance {
let dist_instance = DistInstance::new(
meta_client.clone(),
Arc::new(catalog_manager.clone()),
datanode_clients,
datanode_clients.clone(),
);
let dist_instance = Arc::new(dist_instance);
catalog_manager.set_dist_instance(dist_instance.clone());
let catalog_manager = Arc::new(catalog_manager);
let query_engine =
QueryEngineFactory::new_with_plugins(catalog_manager.clone(), false, plugins.clone())
.query_engine();
let query_engine = QueryEngineFactory::new_with_plugins(
catalog_manager.clone(),
true,
Some(partition_manager.clone()),
Some(datanode_clients),
plugins.clone(),
)
.query_engine();
let script_executor =
Arc::new(ScriptExecutor::new(catalog_manager.clone(), query_engine.clone()).await?);
@@ -276,24 +282,21 @@ impl Instance {
/// Handle batch inserts
pub async fn handle_inserts(
&self,
requests: Vec<InsertRequest>,
requests: InsertRequests,
ctx: QueryContextRef,
) -> Result<Output> {
let mut success = 0;
for request in requests {
match self.handle_insert(request, ctx.clone()).await? {
Output::AffectedRows(rows) => success += rows,
_ => unreachable!("Insert should not yield output other than AffectedRows"),
}
}
Ok(Output::AffectedRows(success))
}
// TODO(LFC): Optimize concurrent table creation and table alteration.
// Currently table creation is guarded by a distributed lock in Metasrv. However, table
// alteration is not. We should all switch to procedures in Metasrv.
let _ = future::join_all(
requests
.inserts
.iter()
.map(|x| self.create_or_alter_table_on_demand(ctx.clone(), x)),
)
.await;
async fn handle_insert(&self, request: InsertRequest, ctx: QueryContextRef) -> Result<Output> {
self.create_or_alter_table_on_demand(ctx.clone(), &request)
.await?;
let query = Request::Insert(request);
let query = Request::Inserts(requests);
GrpcQueryHandler::do_query(&*self.grpc_query_handler, query, ctx).await
}
@@ -433,7 +436,7 @@ impl Instance {
#[async_trait]
impl FrontendInstance for Instance {
async fn start(&mut self) -> Result<()> {
async fn start(&self) -> Result<()> {
// TODO(hl): Frontend init should move to here
if let Some(heartbeat_task) = &self.heartbeat_task {

View File

@@ -12,6 +12,8 @@
// See the License for the specific language governing permissions and
// limitations under the License.
pub(crate) mod inserter;
use std::collections::HashMap;
use std::sync::Arc;
@@ -20,12 +22,13 @@ use api::v1::ddl_request::Expr as DdlExpr;
use api::v1::greptime_request::Request;
use api::v1::{
column_def, AlterExpr, CreateDatabaseExpr, CreateTableExpr, DeleteRequest, DropTableExpr,
FlushTableExpr, InsertRequest, TableId,
FlushTableExpr, InsertRequests, TableId,
};
use async_trait::async_trait;
use catalog::helper::{SchemaKey, SchemaValue};
use catalog::{CatalogManager, DeregisterTableRequest, RegisterTableRequest};
use chrono::DateTime;
use client::client_manager::DatanodeClients;
use client::Database;
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
use common_catalog::format_full_table_name;
@@ -37,7 +40,7 @@ use common_meta::rpc::router::{
use common_meta::rpc::store::CompareAndPutRequest;
use common_meta::table_name::TableName;
use common_query::Output;
use common_telemetry::debug;
use common_telemetry::{debug, info, warn};
use datanode::instance::sql::table_idents_to_full_name;
use datanode::sql::SqlHandler;
use datatypes::prelude::ConcreteDataType;
@@ -54,22 +57,23 @@ use sql::ast::{Ident, Value as SqlValue};
use sql::statements::create::{PartitionEntry, Partitions};
use sql::statements::statement::Statement;
use sql::statements::{self, sql_value_to_value};
use table::engine::TableReference;
use store_api::storage::RegionNumber;
use table::engine::{self, TableReference};
use table::metadata::{RawTableInfo, RawTableMeta, TableIdent, TableType};
use table::requests::TableOptions;
use table::table::AlterContext;
use table::TableRef;
use crate::catalog::FrontendCatalogManager;
use crate::datanode::DatanodeClients;
use crate::error::{
self, AlterExprToRequestSnafu, CatalogEntrySerdeSnafu, CatalogSnafu, ColumnDataTypeSnafu,
DeserializePartitionSnafu, InvokeDatanodeSnafu, ParseSqlSnafu, PrimaryKeyNotFoundSnafu,
RequestDatanodeSnafu, RequestMetaSnafu, Result, SchemaExistsSnafu, StartMetaClientSnafu,
TableAlreadyExistSnafu, TableNotFoundSnafu, TableSnafu, ToTableDeleteRequestSnafu,
ToTableInsertRequestSnafu, UnrecognizedTableOptionSnafu,
UnrecognizedTableOptionSnafu,
};
use crate::expr_factory;
use crate::instance::distributed::inserter::DistInserter;
use crate::table::DistTable;
const MAX_VALUE: &str = "MAXVALUE";
@@ -94,6 +98,17 @@ impl DistInstance {
}
}
async fn find_table(&self, table_name: &TableName) -> Result<Option<TableRef>> {
self.catalog_manager
.table(
&table_name.catalog_name,
&table_name.schema_name,
&table_name.table_name,
)
.await
.context(CatalogSnafu)
}
pub async fn create_table(
&self,
create_table: &mut CreateTableExpr,
@@ -106,16 +121,7 @@ impl DistInstance {
&create_table.table_name,
);
if let Some(table) = self
.catalog_manager
.table(
&table_name.catalog_name,
&table_name.schema_name,
&table_name.table_name,
)
.await
.context(CatalogSnafu)?
{
if let Some(table) = self.find_table(&table_name).await? {
return if create_table.create_if_not_exists {
Ok(table)
} else {
@@ -130,7 +136,19 @@ impl DistInstance {
let response = self
.create_table_in_meta(create_table, partitions, &table_info)
.await?;
.await;
let response = match response {
Ok(response) => response,
Err(e) => {
return if let Some(table) = self.find_table(&table_name).await? {
warn!("Table '{table_name}' is created concurrently by other Frontend nodes!");
Ok(table)
} else {
Err(e)
}
}
};
let table_routes = response.table_routes;
ensure!(
table_routes.len() == 1,
@@ -139,7 +157,7 @@ impl DistInstance {
}
);
let table_route = table_routes.first().unwrap();
debug!(
info!(
"Creating distributed table {table_name} with table routes: {}",
serde_json::to_string_pretty(table_route)
.unwrap_or_else(|_| format!("{table_route:#?}"))
@@ -161,9 +179,7 @@ impl DistInstance {
let table = Arc::new(DistTable::new(
table_name.clone(),
table_info,
self.catalog_manager.partition_manager(),
self.catalog_manager.datanode_clients(),
self.catalog_manager.backend(),
self.catalog_manager.clone(),
));
let request = RegisterTableRequest {
@@ -189,11 +205,11 @@ impl DistInstance {
let regions = table_route.find_leader_regions(&datanode);
let mut create_expr_for_region = create_table.clone();
create_expr_for_region.region_ids = regions;
create_expr_for_region.region_numbers = regions;
debug!(
"Creating table {:?} on Datanode {:?} with regions {:?}",
create_table, datanode, create_expr_for_region.region_ids,
create_table, datanode, create_expr_for_region.region_numbers,
);
let _timer = common_telemetry::timer!(crate::metrics::DIST_CREATE_TABLE_IN_DATANODE);
@@ -288,7 +304,11 @@ impl DistInstance {
Ok(Output::AffectedRows(1))
}
async fn flush_table(&self, table_name: TableName, region_id: Option<u32>) -> Result<Output> {
async fn flush_table(
&self,
table_name: TableName,
region_number: Option<RegionNumber>,
) -> Result<Output> {
let _ = self
.catalog_manager
.table(
@@ -314,13 +334,13 @@ impl DistInstance {
catalog_name: table_name.catalog_name.clone(),
schema_name: table_name.schema_name.clone(),
table_name: table_name.table_name.clone(),
region_id,
region_number,
};
for table_route in &route_response.table_routes {
let should_send_rpc = table_route.region_routes.iter().any(|route| {
if let Some(region_id) = region_id {
region_id == route.region.id as u32
if let Some(n) = region_number {
n == engine::region_number(route.region.id)
} else {
true
}
@@ -564,36 +584,21 @@ impl DistInstance {
.context(RequestMetaSnafu)
}
// TODO(LFC): Refactor GRPC insertion and deletion implementation here,
// Take insertion as an example. GRPC insertion is converted to Table InsertRequest here,
// than split the Table InsertRequest in DistTable, than assemble each GRPC InsertRequest there.
// Rather inefficient, should operate on GRPC InsertRequest directly.
async fn handle_dist_insert(
&self,
request: InsertRequest,
requests: InsertRequests,
ctx: QueryContextRef,
) -> Result<Output> {
let catalog = &ctx.current_catalog();
let schema = &ctx.current_schema();
let table_name = &request.table_name;
let table_ref = TableReference::full(catalog, schema, table_name);
let table = self
.catalog_manager
.table(catalog, schema, table_name)
.await
.context(CatalogSnafu)?
.with_context(|| TableNotFoundSnafu {
table_name: table_ref.to_string(),
})?;
let request = common_grpc_expr::insert::to_table_insert_request(catalog, schema, request)
.context(ToTableInsertRequestSnafu)?;
let affected_rows = table.insert(request).await.context(TableSnafu)?;
Ok(Output::AffectedRows(affected_rows))
let inserter = DistInserter::new(
ctx.current_catalog(),
ctx.current_schema(),
self.catalog_manager.clone(),
);
let affected_rows = inserter.grpc_insert(requests).await?;
Ok(Output::AffectedRows(affected_rows as usize))
}
// TODO(LFC): Like insertions above, refactor GRPC deletion impl here.
async fn handle_dist_delete(
&self,
request: DeleteRequest,
@@ -645,7 +650,7 @@ impl GrpcQueryHandler for DistInstance {
async fn do_query(&self, request: Request, ctx: QueryContextRef) -> Result<Output> {
match request {
Request::Insert(request) => self.handle_dist_insert(request, ctx).await,
Request::Inserts(requests) => self.handle_dist_insert(requests, ctx).await,
Request::Delete(request) => self.handle_dist_delete(request, ctx).await,
Request::Query(_) => {
unreachable!("Query should have been handled directly in Frontend Instance!")
@@ -671,7 +676,7 @@ impl GrpcQueryHandler for DistInstance {
DdlExpr::FlushTable(expr) => {
let table_name =
TableName::new(&expr.catalog_name, &expr.schema_name, &expr.table_name);
self.flush_table(table_name, expr.region_id).await
self.flush_table(table_name, expr.region_number).await
}
}
}

View File

@@ -0,0 +1,380 @@
// 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 api::v1::InsertRequests;
use catalog::CatalogManager;
use client::Database;
use common_grpc_expr::insert::to_table_insert_request;
use common_meta::peer::Peer;
use common_meta::table_name::TableName;
use futures::future;
use metrics::counter;
use snafu::{OptionExt, ResultExt};
use table::metadata::TableInfoRef;
use table::meter_insert_request;
use table::requests::InsertRequest;
use crate::catalog::FrontendCatalogManager;
use crate::error::{
CatalogSnafu, FindDatanodeSnafu, FindTableRouteSnafu, JoinTaskSnafu, RequestDatanodeSnafu,
Result, SplitInsertSnafu, TableNotFoundSnafu, ToTableInsertRequestSnafu,
};
use crate::table::insert::to_grpc_insert_request;
/// A distributed inserter. It ingests GRPC [InsertRequests] or table [InsertRequest] (so it can be
/// used in protocol handlers or table insertion API).
///
/// Table data partitioning and Datanode requests batching are handled inside.
///
/// Note that the inserter is confined to a single catalog and schema. I.e., it cannot handle
/// multiple insert requests with different catalog or schema (will throw "NotSupported" error).
/// This is because we currently do not have this kind of requirements. Let's keep it simple for now.
pub(crate) struct DistInserter {
catalog: String,
schema: String,
catalog_manager: Arc<FrontendCatalogManager>,
}
impl DistInserter {
pub(crate) fn new(
catalog: String,
schema: String,
catalog_manager: Arc<FrontendCatalogManager>,
) -> Self {
Self {
catalog,
schema,
catalog_manager,
}
}
pub(crate) async fn grpc_insert(&self, requests: InsertRequests) -> Result<u32> {
let inserts = requests
.inserts
.into_iter()
.map(|x| {
to_table_insert_request(&self.catalog, &self.schema, x)
.context(ToTableInsertRequestSnafu)
})
.collect::<Result<Vec<_>>>()?;
self.insert(inserts).await
}
pub(crate) async fn insert(&self, requests: Vec<InsertRequest>) -> Result<u32> {
debug_assert!(requests
.iter()
.all(|x| x.catalog_name == self.catalog && x.schema_name == self.schema));
let inserts = self.split_inserts(requests).await?;
self.request_datanodes(inserts).await
}
/// Splits multiple table [InsertRequest]s into multiple GRPC [InsertRequests]s, each of which
/// is grouped by the peer of Datanode, so we can batch them together when invoking gRPC write
/// method in Datanode.
async fn split_inserts(
&self,
requests: Vec<InsertRequest>,
) -> Result<HashMap<Peer, InsertRequests>> {
let partition_manager = self.catalog_manager.partition_manager();
let mut inserts = HashMap::new();
for request in requests {
meter_insert_request!(request);
let table_name = TableName::new(&self.catalog, &self.schema, &request.table_name);
let table_info = self.find_table_info(&request.table_name).await?;
let table_meta = &table_info.meta;
let split = partition_manager
.split_insert_request(&table_name, request, table_meta.schema.as_ref())
.await
.context(SplitInsertSnafu)?;
let table_route = partition_manager
.find_table_route(&table_name)
.await
.with_context(|_| FindTableRouteSnafu {
table_name: table_name.to_string(),
})?;
for (region_number, insert) in split {
let datanode =
table_route
.find_region_leader(region_number)
.context(FindDatanodeSnafu {
region: region_number,
})?;
let insert = to_grpc_insert_request(table_meta, region_number, insert)?;
inserts
.entry(datanode.clone())
.or_insert_with(|| InsertRequests { inserts: vec![] })
.inserts
.push(insert);
}
}
Ok(inserts)
}
async fn find_table_info(&self, table_name: &str) -> Result<TableInfoRef> {
let table = self
.catalog_manager
.table(&self.catalog, &self.schema, table_name)
.await
.context(CatalogSnafu)?
.with_context(|| TableNotFoundSnafu {
table_name: common_catalog::format_full_table_name(
&self.catalog,
&self.schema,
table_name,
),
})?;
Ok(table.table_info())
}
async fn request_datanodes(&self, inserts: HashMap<Peer, InsertRequests>) -> Result<u32> {
let results = future::try_join_all(inserts.into_iter().map(|(peer, inserts)| {
let datanode_clients = self.catalog_manager.datanode_clients();
let catalog = self.catalog.clone();
let schema = self.schema.clone();
common_runtime::spawn_write(async move {
let client = datanode_clients.get_client(&peer).await;
let database = Database::new(&catalog, &schema, client);
database.insert(inserts).await.context(RequestDatanodeSnafu)
})
}))
.await
.context(JoinTaskSnafu)?;
let affected_rows = results.into_iter().sum::<Result<u32>>()?;
counter!(crate::metrics::DIST_INGEST_ROW_COUNT, affected_rows as u64);
Ok(affected_rows)
}
}
#[cfg(test)]
mod tests {
use api::v1::column::{SemanticType, Values};
use api::v1::{Column, ColumnDataType, InsertRequest as GrpcInsertRequest};
use catalog::helper::{
CatalogKey, CatalogValue, SchemaKey, SchemaValue, TableGlobalKey, TableGlobalValue,
};
use catalog::remote::mock::MockKvBackend;
use catalog::remote::{KvBackend, KvBackendRef};
use client::client_manager::DatanodeClients;
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
use datatypes::prelude::{ConcreteDataType, VectorRef};
use datatypes::schema::{ColumnDefaultConstraint, ColumnSchema, Schema};
use datatypes::vectors::Int32Vector;
use table::metadata::{TableInfoBuilder, TableMetaBuilder};
use super::*;
use crate::heartbeat::handler::tests::MockKvCacheInvalidator;
use crate::table::test::create_partition_rule_manager;
async fn prepare_mocked_backend() -> KvBackendRef {
let backend = Arc::new(MockKvBackend::default());
let default_catalog = CatalogKey {
catalog_name: DEFAULT_CATALOG_NAME.to_string(),
}
.to_string();
backend
.set(
default_catalog.as_bytes(),
CatalogValue.as_bytes().unwrap().as_slice(),
)
.await
.unwrap();
let default_schema = SchemaKey {
catalog_name: DEFAULT_CATALOG_NAME.to_string(),
schema_name: DEFAULT_SCHEMA_NAME.to_string(),
}
.to_string();
backend
.set(
default_schema.as_bytes(),
SchemaValue.as_bytes().unwrap().as_slice(),
)
.await
.unwrap();
backend
}
async fn create_testing_table(backend: &KvBackendRef, table_name: &str) {
let table_global_key = TableGlobalKey {
catalog_name: DEFAULT_CATALOG_NAME.to_string(),
schema_name: DEFAULT_SCHEMA_NAME.to_string(),
table_name: table_name.to_string(),
};
let schema = Arc::new(Schema::new(vec![
ColumnSchema::new("ts", ConcreteDataType::int64_datatype(), false)
.with_time_index(true)
.with_default_constraint(Some(ColumnDefaultConstraint::Function(
"current_timestamp()".to_string(),
)))
.unwrap(),
ColumnSchema::new("a", ConcreteDataType::int32_datatype(), true),
]));
let mut builder = TableMetaBuilder::default();
builder.schema(schema);
builder.primary_key_indices(vec![]);
builder.next_column_id(1);
let table_meta = builder.build().unwrap();
let table_info = TableInfoBuilder::new(table_name, table_meta)
.build()
.unwrap();
let table_global_value = TableGlobalValue {
node_id: 1,
regions_id_map: HashMap::from([(1, vec![1]), (2, vec![2]), (3, vec![3])]),
table_info: table_info.into(),
};
backend
.set(
table_global_key.to_string().as_bytes(),
table_global_value.as_bytes().unwrap().as_slice(),
)
.await
.unwrap();
}
#[tokio::test]
async fn test_split_inserts() {
let backend = prepare_mocked_backend().await;
let table_name = "one_column_partitioning_table";
create_testing_table(&backend, table_name).await;
let catalog_manager = Arc::new(FrontendCatalogManager::new(
backend,
Arc::new(MockKvCacheInvalidator::default()),
create_partition_rule_manager().await,
Arc::new(DatanodeClients::default()),
));
let inserter = DistInserter::new(
DEFAULT_CATALOG_NAME.to_string(),
DEFAULT_SCHEMA_NAME.to_string(),
catalog_manager,
);
let new_insert_request = |vector: VectorRef| -> InsertRequest {
InsertRequest {
catalog_name: DEFAULT_CATALOG_NAME.to_string(),
schema_name: DEFAULT_SCHEMA_NAME.to_string(),
table_name: table_name.to_string(),
columns_values: HashMap::from([("a".to_string(), vector)]),
region_number: 0,
}
};
let requests = vec![
new_insert_request(Arc::new(Int32Vector::from(vec![
Some(1),
None,
Some(11),
Some(101),
]))),
new_insert_request(Arc::new(Int32Vector::from(vec![
Some(2),
Some(12),
None,
Some(102),
]))),
];
let mut inserts = inserter.split_inserts(requests).await.unwrap();
assert_eq!(inserts.len(), 3);
let new_grpc_insert_request = |column_values: Vec<i32>,
null_mask: Vec<u8>,
row_count: u32,
region_number: u32|
-> GrpcInsertRequest {
GrpcInsertRequest {
table_name: table_name.to_string(),
columns: vec![Column {
column_name: "a".to_string(),
semantic_type: SemanticType::Field as i32,
values: Some(Values {
i32_values: column_values,
..Default::default()
}),
null_mask,
datatype: ColumnDataType::Int32 as i32,
}],
row_count,
region_number,
}
};
// region to datanode placement:
// 1 -> 1
// 2 -> 2
// 3 -> 3
//
// region value ranges:
// 1 -> [50, max)
// 2 -> [10, 50)
// 3 -> (min, 10)
let datanode_inserts = inserts.remove(&Peer::new(1, "")).unwrap().inserts;
assert_eq!(datanode_inserts.len(), 2);
assert_eq!(
datanode_inserts[0],
new_grpc_insert_request(vec![101], vec![0], 1, 1)
);
assert_eq!(
datanode_inserts[1],
new_grpc_insert_request(vec![102], vec![0], 1, 1)
);
let datanode_inserts = inserts.remove(&Peer::new(2, "")).unwrap().inserts;
assert_eq!(datanode_inserts.len(), 2);
assert_eq!(
datanode_inserts[0],
new_grpc_insert_request(vec![11], vec![0], 1, 2)
);
assert_eq!(
datanode_inserts[1],
new_grpc_insert_request(vec![12], vec![0], 1, 2)
);
let datanode_inserts = inserts.remove(&Peer::new(3, "")).unwrap().inserts;
assert_eq!(datanode_inserts.len(), 2);
assert_eq!(
datanode_inserts[0],
new_grpc_insert_request(vec![1], vec![2], 2, 3)
);
assert_eq!(
datanode_inserts[1],
new_grpc_insert_request(vec![2], vec![2], 2, 3)
);
}
}

View File

@@ -36,7 +36,7 @@ impl GrpcQueryHandler for Instance {
interceptor.pre_execute(&request, ctx.clone())?;
let output = match request {
Request::Insert(request) => self.handle_insert(request, ctx.clone()).await?,
Request::Inserts(requests) => self.handle_inserts(requests, ctx.clone()).await?,
Request::Query(query_request) => {
let query = query_request.query.context(IncompleteGrpcResultSnafu {
err_msg: "Missing field 'QueryRequest.query'",

View File

@@ -12,6 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use api::v1::InsertRequests;
use async_trait::async_trait;
use common_error::prelude::BoxedError;
use servers::error as server_error;
@@ -25,8 +26,10 @@ use crate::instance::Instance;
#[async_trait]
impl OpentsdbProtocolHandler for Instance {
async fn exec(&self, data_point: &DataPoint, ctx: QueryContextRef) -> server_error::Result<()> {
let request = data_point.as_grpc_insert();
self.handle_insert(request, ctx)
let requests = InsertRequests {
inserts: vec![data_point.as_grpc_insert()],
};
self.handle_inserts(requests, ctx)
.await
.map_err(BoxedError::new)
.with_context(|_| server_error::ExecuteQuerySnafu {

View File

@@ -21,6 +21,7 @@ use common_error::prelude::BoxedError;
use common_query::Output;
use common_recordbatch::RecordBatches;
use common_telemetry::logging;
use metrics::counter;
use prost::Message;
use servers::error::{self, Result as ServerResult};
use servers::prometheus::{self, Metrics};
@@ -30,6 +31,7 @@ use session::context::QueryContextRef;
use snafu::{OptionExt, ResultExt};
use crate::instance::Instance;
use crate::metrics::PROMETHEUS_REMOTE_WRITE_SAMPLES;
const SAMPLES_RESPONSE_TYPE: i32 = ResponseType::Samples as i32;
@@ -106,11 +108,13 @@ impl Instance {
#[async_trait]
impl PrometheusProtocolHandler for Instance {
async fn write(&self, request: WriteRequest, ctx: QueryContextRef) -> ServerResult<()> {
let requests = prometheus::to_grpc_insert_requests(request.clone())?;
let (requests, samples) = prometheus::to_grpc_insert_requests(request)?;
self.handle_inserts(requests, ctx)
.await
.map_err(BoxedError::new)
.context(error::ExecuteGrpcQuerySnafu)?;
counter!(PROMETHEUS_REMOTE_WRITE_SAMPLES, samples as u64);
Ok(())
}

View File

@@ -16,7 +16,6 @@
#![feature(trait_upcasting)]
pub mod catalog;
pub mod datanode;
pub mod error;
pub mod expr_factory;
pub mod frontend;

View File

@@ -22,3 +22,6 @@ pub const DIST_CREATE_TABLE: &str = "frontend.dist.create_table";
pub const DIST_CREATE_TABLE_IN_META: &str = "frontend.dist.create_table.update_meta";
pub const DIST_CREATE_TABLE_IN_DATANODE: &str = "frontend.dist.create_table.invoke_datanode";
pub const DIST_INGEST_ROW_COUNT: &str = "frontend.dist.ingest_rows";
/// The samples count of Prometheus remote write.
pub const PROMETHEUS_REMOTE_WRITE_SAMPLES: &str = "frontend.prometheus.remote_write.samples";

View File

@@ -20,9 +20,9 @@ use std::sync::Arc;
use api::v1::AlterExpr;
use async_trait::async_trait;
use catalog::helper::{TableGlobalKey, TableGlobalValue};
use catalog::remote::KvBackendRef;
use client::Database;
use common_error::prelude::BoxedError;
use common_meta::key::TableRouteKey;
use common_meta::table_name::TableName;
use common_query::error::Result as QueryResult;
use common_query::logical_plan::Expr;
@@ -43,7 +43,7 @@ use datafusion::physical_plan::{
use datafusion_common::DataFusionError;
use datatypes::schema::{ColumnSchema, Schema, SchemaRef};
use futures_util::{Stream, StreamExt};
use partition::manager::PartitionRuleManagerRef;
use partition::manager::TableRouteCacheInvalidator;
use partition::splitter::WriteSplitter;
use snafu::prelude::*;
use store_api::storage::{RegionNumber, ScanRequest};
@@ -51,13 +51,13 @@ use table::error::TableOperationSnafu;
use table::metadata::{FilterPushDownType, TableInfo, TableInfoRef};
use table::requests::{AlterKind, AlterTableRequest, DeleteRequest, InsertRequest};
use table::table::AlterContext;
use table::{meter_insert_request, Table};
use table::Table;
use tokio::sync::RwLock;
use crate::datanode::DatanodeClients;
use crate::catalog::FrontendCatalogManager;
use crate::error::{self, FindDatanodeSnafu, FindTableRouteSnafu, Result};
use crate::instance::distributed::inserter::DistInserter;
use crate::table::delete::to_grpc_delete_request;
use crate::table::insert::to_grpc_insert_request;
use crate::table::scan::{DatanodeInstance, TableScanPlan};
mod delete;
@@ -68,9 +68,7 @@ pub(crate) mod scan;
pub struct DistTable {
table_name: TableName,
table_info: TableInfoRef,
partition_manager: PartitionRuleManagerRef,
datanode_clients: Arc<DatanodeClients>,
backend: KvBackendRef,
catalog_manager: Arc<FrontendCatalogManager>,
}
#[async_trait]
@@ -88,29 +86,17 @@ impl Table for DistTable {
}
async fn insert(&self, request: InsertRequest) -> table::Result<usize> {
meter_insert_request!(request);
let splits = self
.partition_manager
.split_insert_request(&self.table_name, request, &self.schema())
let inserter = DistInserter::new(
request.catalog_name.clone(),
request.schema_name.clone(),
self.catalog_manager.clone(),
);
let affected_rows = inserter
.insert(vec![request])
.await
.map_err(BoxedError::new)
.context(TableOperationSnafu)?;
let inserts = splits
.into_iter()
.map(|(region_number, insert)| to_grpc_insert_request(region_number, insert))
.collect::<Result<Vec<_>>>()
.map_err(BoxedError::new)
.context(TableOperationSnafu)?;
let output = self
.dist_insert(inserts)
.await
.map_err(BoxedError::new)
.context(TableOperationSnafu)?;
let Output::AffectedRows(rows) = output else { unreachable!() };
Ok(rows)
Ok(affected_rows as usize)
}
async fn scan(
@@ -119,20 +105,20 @@ impl Table for DistTable {
filters: &[Expr],
limit: Option<usize>,
) -> table::Result<PhysicalPlanRef> {
let partition_rule = self
.partition_manager
let partition_manager = self.catalog_manager.partition_manager();
let datanode_clients = self.catalog_manager.datanode_clients();
let partition_rule = partition_manager
.find_table_partition_rule(&self.table_name)
.await
.map_err(BoxedError::new)
.context(TableOperationSnafu)?;
let regions = self
.partition_manager
let regions = partition_manager
.find_regions_by_filters(partition_rule, filters)
.map_err(BoxedError::new)
.context(TableOperationSnafu)?;
let datanodes = self
.partition_manager
let datanodes = partition_manager
.find_region_datanodes(&self.table_name, regions)
.await
.map_err(BoxedError::new)
@@ -141,7 +127,7 @@ impl Table for DistTable {
let table_name = &self.table_name;
let mut partition_execs = Vec::with_capacity(datanodes.len());
for (datanode, _regions) in datanodes.iter() {
let client = self.datanode_clients.get_client(datanode).await;
let client = datanode_clients.get_client(datanode).await;
let db = Database::new(&table_name.catalog_name, &table_name.schema_name, client);
let datanode_instance = DatanodeInstance::new(Arc::new(self.clone()) as _, db);
@@ -167,20 +153,20 @@ impl Table for DistTable {
&self,
request: ScanRequest,
) -> table::Result<SendableRecordBatchStream> {
let partition_rule = self
.partition_manager
let partition_manager = self.catalog_manager.partition_manager();
let datanode_clients = self.catalog_manager.datanode_clients();
let partition_rule = partition_manager
.find_table_partition_rule(&self.table_name)
.await
.map_err(BoxedError::new)
.context(TableOperationSnafu)?;
let regions = self
.partition_manager
let regions = partition_manager
.find_regions_by_filters(partition_rule, &request.filters)
.map_err(BoxedError::new)
.context(TableOperationSnafu)?;
let datanodes = self
.partition_manager
let datanodes = partition_manager
.find_region_datanodes(&self.table_name, regions)
.await
.map_err(BoxedError::new)
@@ -189,7 +175,7 @@ impl Table for DistTable {
let table_name = &self.table_name;
let mut partition_execs = Vec::with_capacity(datanodes.len());
for (datanode, _regions) in datanodes.iter() {
let client = self.datanode_clients.get_client(datanode).await;
let client = datanode_clients.get_client(datanode).await;
let db = Database::new(&table_name.catalog_name, &table_name.schema_name, client);
let datanode_instance = DatanodeInstance::new(Arc::new(self.clone()) as _, db);
@@ -221,7 +207,11 @@ impl Table for DistTable {
}
},
);
let record_batch_stream = RecordBatchStreamAdaptor { schema, stream };
let record_batch_stream = RecordBatchStreamAdaptor {
schema,
stream,
output_ordering: None,
};
Ok(Box::pin(record_batch_stream))
}
@@ -241,8 +231,9 @@ impl Table for DistTable {
}
async fn delete(&self, request: DeleteRequest) -> table::Result<usize> {
let partition_rule = self
.partition_manager
let partition_manager = self.catalog_manager.partition_manager();
let partition_rule = partition_manager
.find_table_partition_rule(&self.table_name)
.await
.map_err(BoxedError::new)
@@ -270,7 +261,12 @@ impl Table for DistTable {
requests
.into_iter()
.map(|(region_number, request)| {
to_grpc_delete_request(&self.table_name, region_number, request)
to_grpc_delete_request(
&table_info.meta,
&self.table_name,
region_number,
request,
)
})
.collect::<Result<Vec<_>>>()
.map_err(BoxedError::new)
@@ -291,16 +287,12 @@ impl DistTable {
pub fn new(
table_name: TableName,
table_info: TableInfoRef,
partition_manager: PartitionRuleManagerRef,
datanode_clients: Arc<DatanodeClients>,
backend: KvBackendRef,
catalog_manager: Arc<FrontendCatalogManager>,
) -> Self {
Self {
table_name,
table_info,
partition_manager,
datanode_clients,
backend,
catalog_manager,
}
}
@@ -309,7 +301,8 @@ impl DistTable {
key: &TableGlobalKey,
) -> Result<Option<TableGlobalValue>> {
let raw = self
.backend
.catalog_manager
.backend()
.get(key.to_string().as_bytes())
.await
.context(error::CatalogSnafu)?;
@@ -326,20 +319,71 @@ impl DistTable {
value: TableGlobalValue,
) -> Result<()> {
let value = value.as_bytes().context(error::CatalogEntrySerdeSnafu)?;
self.backend
self.catalog_manager
.backend()
.set(key.to_string().as_bytes(), &value)
.await
.context(error::CatalogSnafu)
}
async fn delete_table_global_value(&self, key: TableGlobalKey) -> Result<()> {
self.backend
self.catalog_manager
.backend()
.delete(key.to_string().as_bytes())
.await
.context(error::CatalogSnafu)
}
async fn move_table_route_value(
&self,
catalog_name: &str,
schema_name: &str,
table_id: u32,
old_table_name: &str,
new_table_name: &str,
) -> Result<()> {
let old_key = TableRouteKey {
table_id: table_id.into(),
catalog_name,
schema_name,
table_name: old_table_name,
}
.key();
let new_key = TableRouteKey {
table_id: table_id.into(),
catalog_name,
schema_name,
table_name: new_table_name,
}
.key();
self.catalog_manager
.backend()
.move_value(old_key.as_bytes(), new_key.as_bytes())
.await
.context(error::CatalogSnafu)?;
self.catalog_manager
.partition_manager()
.invalidate_table_route(&TableName {
catalog_name: catalog_name.to_string(),
schema_name: schema_name.to_string(),
table_name: old_table_name.to_string(),
})
.await;
Ok(())
}
async fn handle_alter(&self, context: AlterContext, request: &AlterTableRequest) -> Result<()> {
let AlterTableRequest {
catalog_name,
schema_name,
table_name,
alter_kind,
} = request;
let alter_expr = context
.get::<AlterExpr>()
.context(error::ContextValueNotFoundSnafu { key: "AlterExpr" })?;
@@ -347,7 +391,6 @@ impl DistTable {
self.alter_by_expr(alter_expr).await?;
let table_info = self.table_info();
let table_name = &table_info.name;
let new_meta = table_info
.meta
.builder_with_alter_kind(table_name, &request.alter_kind)
@@ -362,27 +405,34 @@ impl DistTable {
new_info.meta = new_meta;
let key = TableGlobalKey {
catalog_name: alter_expr.catalog_name.clone(),
schema_name: alter_expr.schema_name.clone(),
table_name: alter_expr.table_name.clone(),
catalog_name: catalog_name.clone(),
schema_name: schema_name.clone(),
table_name: table_name.clone(),
};
let mut value =
self.table_global_value(&key)
.await?
.context(error::TableNotFoundSnafu {
table_name: alter_expr.table_name.clone(),
})?;
let mut value = self
.table_global_value(&key)
.await?
.context(error::TableNotFoundSnafu { table_name })?;
value.table_info = new_info.into();
if let AlterKind::RenameTable { new_table_name } = &request.alter_kind {
if let AlterKind::RenameTable { new_table_name } = alter_kind {
let new_key = TableGlobalKey {
catalog_name: alter_expr.catalog_name.clone(),
schema_name: alter_expr.schema_name.clone(),
catalog_name: catalog_name.clone(),
schema_name: schema_name.clone(),
table_name: new_table_name.clone(),
};
self.set_table_global_value(new_key, value).await?;
self.delete_table_global_value(key).await
self.delete_table_global_value(key).await?;
self.move_table_route_value(
catalog_name,
schema_name,
table_info.ident.table_id,
table_name,
new_table_name,
)
.await?;
Ok(())
} else {
self.set_table_global_value(key, value).await
}
@@ -392,7 +442,8 @@ impl DistTable {
/// [`table::requests::AlterTableRequest`] and [`AlterExpr`].
async fn alter_by_expr(&self, expr: &AlterExpr) -> Result<()> {
let table_routes = self
.partition_manager
.catalog_manager
.partition_manager()
.find_table_route(&self.table_name)
.await
.with_context(|_| error::FindTableRouteSnafu {
@@ -408,8 +459,10 @@ impl DistTable {
)
}
);
let datanode_clients = self.catalog_manager.datanode_clients();
for datanode in leaders {
let client = self.datanode_clients.get_client(&datanode).await;
let client = datanode_clients.get_client(&datanode).await;
let db = Database::new(&expr.catalog_name, &expr.schema_name, client);
debug!("Sending {:?} to {:?}", expr, db);
let result = db
@@ -428,7 +481,8 @@ impl DistTable {
) -> Result<Vec<DatanodeInstance>> {
let table_name = &self.table_name;
let route = self
.partition_manager
.catalog_manager
.partition_manager()
.find_table_route(table_name)
.await
.with_context(|_| FindTableRouteSnafu {
@@ -438,24 +492,16 @@ impl DistTable {
let datanodes = regions
.iter()
.map(|&n| {
let region_id = n as u64;
route
.region_routes
.iter()
.find_map(|x| {
if x.region.id == region_id {
x.leader_peer.clone()
} else {
None
}
})
.context(FindDatanodeSnafu { region: region_id })
.find_region_leader(n)
.context(FindDatanodeSnafu { region: n })
})
.collect::<Result<Vec<_>>>()?;
let datanode_clients = self.catalog_manager.datanode_clients();
let mut instances = Vec::with_capacity(datanodes.len());
for datanode in datanodes {
let client = self.datanode_clients.get_client(&datanode).await;
let client = datanode_clients.get_client(datanode).await;
let db = Database::new(&table_name.catalog_name, &table_name.schema_name, client);
instances.push(DatanodeInstance::new(Arc::new(self.clone()) as _, db));
}
@@ -541,7 +587,7 @@ impl PartitionExec {
return Ok(());
}
let plan = TableScanPlan {
let plan: TableScanPlan = TableScanPlan {
table_name: self.table_name.clone(),
projection: self.projection.clone(),
filters: self.filters.clone(),
@@ -563,12 +609,12 @@ impl PartitionExec {
}
#[cfg(test)]
mod test {
pub(crate) mod test {
use std::collections::HashMap;
use std::sync::atomic::{AtomicU32, Ordering};
use catalog::error::Result;
use catalog::remote::{KvBackend, ValueIter};
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
use common_meta::peer::Peer;
use common_meta::rpc::router::{Region, RegionRoute, Table, TableRoute};
use datafusion_expr::expr_fn::{and, binary_expr, col, or};
use datafusion_expr::{lit, Operator};
@@ -578,7 +624,7 @@ mod test {
use meter_core::global::global_registry;
use meter_core::write_calc::WriteCalculator;
use partition::columns::RangeColumnsPartitionRule;
use partition::manager::PartitionRuleManager;
use partition::manager::{PartitionRuleManager, PartitionRuleManagerRef};
use partition::partition::{PartitionBound, PartitionDef};
use partition::range::RangePartitionRule;
use partition::route::TableRoutes;
@@ -588,48 +634,38 @@ mod test {
use super::*;
struct DummyKvBackend;
#[async_trait]
impl KvBackend for DummyKvBackend {
fn range<'a, 'b>(&'a self, _key: &[u8]) -> ValueIter<'b, catalog::error::Error>
where
'a: 'b,
{
unimplemented!()
}
async fn set(&self, _key: &[u8], _val: &[u8]) -> Result<()> {
unimplemented!()
}
async fn compare_and_set(
&self,
_key: &[u8],
_expect: &[u8],
_val: &[u8],
) -> Result<std::result::Result<(), Option<Vec<u8>>>> {
unimplemented!()
}
async fn delete_range(&self, _key: &[u8], _end: &[u8]) -> Result<()> {
unimplemented!()
}
}
#[tokio::test(flavor = "multi_thread")]
async fn test_find_partition_rule() {
let table_name = TableName::new("greptime", "public", "foo");
/// Create a partition rule manager with two tables, one is partitioned by single column, and
/// the other one is two. The tables are under default catalog and schema.
///
/// Table named "one_column_partitioning_table" is partitioned by column "a" like this:
/// PARTITION BY RANGE (a) (
/// PARTITION r1 VALUES LESS THAN (10),
/// PARTITION r2 VALUES LESS THAN (50),
/// PARTITION r3 VALUES LESS THAN (MAXVALUE),
/// )
///
/// Table named "two_column_partitioning_table" is partitioned by columns "a" and "b" like this:
/// PARTITION BY RANGE (a, b) (
/// PARTITION r1 VALUES LESS THAN (10, 'hz'),
/// PARTITION r2 VALUES LESS THAN (50, 'sh'),
/// PARTITION r3 VALUES LESS THAN (MAXVALUE, MAXVALUE),
/// )
pub(crate) async fn create_partition_rule_manager() -> PartitionRuleManagerRef {
let table_routes = Arc::new(TableRoutes::new(Arc::new(MetaClient::default())));
let partition_manager = Arc::new(PartitionRuleManager::new(table_routes.clone()));
let table_route = TableRoute {
table: Table {
let table_name = TableName::new(
DEFAULT_CATALOG_NAME,
DEFAULT_SCHEMA_NAME,
"one_column_partitioning_table",
);
let table_route = TableRoute::new(
Table {
id: 1,
table_name: table_name.clone(),
table_schema: vec![],
},
region_routes: vec![
vec![
RegionRoute {
region: Region {
id: 3,
@@ -644,7 +680,7 @@ mod test {
),
attrs: HashMap::new(),
},
leader_peer: None,
leader_peer: Some(Peer::new(3, "")),
follower_peers: vec![],
},
RegionRoute {
@@ -661,7 +697,7 @@ mod test {
),
attrs: HashMap::new(),
},
leader_peer: None,
leader_peer: Some(Peer::new(2, "")),
follower_peers: vec![],
},
RegionRoute {
@@ -678,34 +714,27 @@ mod test {
),
attrs: HashMap::new(),
},
leader_peer: None,
leader_peer: Some(Peer::new(1, "")),
follower_peers: vec![],
},
],
};
);
table_routes
.insert_table_route(table_name.clone(), Arc::new(table_route))
.await;
let partition_rule = partition_manager
.find_table_partition_rule(&table_name)
.await
.unwrap();
let range_rule = partition_rule
.as_any()
.downcast_ref::<RangePartitionRule>()
.unwrap();
assert_eq!(range_rule.column_name(), "a");
assert_eq!(range_rule.all_regions(), &vec![3, 2, 1]);
assert_eq!(range_rule.bounds(), &vec![10_i32.into(), 50_i32.into()]);
let table_route = TableRoute {
table: Table {
let table_name = TableName::new(
DEFAULT_CATALOG_NAME,
DEFAULT_SCHEMA_NAME,
"two_column_partitioning_table",
);
let table_route = TableRoute::new(
Table {
id: 1,
table_name: table_name.clone(),
table_schema: vec![],
},
region_routes: vec![
vec![
RegionRoute {
region: Region {
id: 1,
@@ -764,13 +793,40 @@ mod test {
follower_peers: vec![],
},
],
};
);
table_routes
.insert_table_route(table_name.clone(), Arc::new(table_route))
.await;
partition_manager
}
#[tokio::test(flavor = "multi_thread")]
async fn test_find_partition_rule() {
let partition_manager = create_partition_rule_manager().await;
let partition_rule = partition_manager
.find_table_partition_rule(&table_name)
.find_table_partition_rule(&TableName::new(
DEFAULT_CATALOG_NAME,
DEFAULT_SCHEMA_NAME,
"one_column_partitioning_table",
))
.await
.unwrap();
let range_rule = partition_rule
.as_any()
.downcast_ref::<RangePartitionRule>()
.unwrap();
assert_eq!(range_rule.column_name(), "a");
assert_eq!(range_rule.all_regions(), &vec![3, 2, 1]);
assert_eq!(range_rule.bounds(), &vec![10_i32.into(), 50_i32.into()]);
let partition_rule = partition_manager
.find_table_partition_rule(&TableName::new(
DEFAULT_CATALOG_NAME,
DEFAULT_SCHEMA_NAME,
"two_column_partitioning_table",
))
.await
.unwrap();
let range_columns_rule = partition_rule

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