Compare commits

...

14 Commits

Author SHA1 Message Date
dennis zhuang
b4fc8c5b78 refactor: make sql function in scripts return a list of column vectors (#1243) 2023-03-27 08:50:19 +08:00
Lei, HUANG
6f81717866 fix: skip empty parquet (#1236)
* fix: returns None if parquet file does not contain any rows

* fix: skip empty parquet file

* chore: add doc

* rebase develop

* fix: use flatten instead of filter_map with identity
2023-03-26 09:39:15 +08:00
Lei, HUANG
77f9383daf fix: allow larger compaction window to reduce parallel task num (#1223)
fix: unit tests
2023-03-24 17:12:13 +08:00
discord9
c788b7fc26 feat: slicing PyVector&Create DataFrame from sql (#1190)
* chore: some typos

* feat: slicing for pyo3 vector

* feat: slice tests

* feat: from_sql

* feat: from_sql for dataframe

* test: df tests

* feat: `from_sql` for rspython

* test: tweak a bit

* test: and CR advices

* typos: ordered points

* chore: update error msg

* test: add more `slicing` testcase
2023-03-24 15:37:45 +08:00
LFC
0f160a73be feat: metasrv collects datanode heartbeats for region failure detection (#1214)
* feat: metasrv collects datanode heartbeats for region failure detection

* chore: change visibility

* fix: fragile tests

* Update src/meta-srv/src/handler/persist_stats_handler.rs

Co-authored-by: fys <40801205+Fengys123@users.noreply.github.com>

* Update src/meta-srv/src/handler/failure_handler.rs

Co-authored-by: fys <40801205+Fengys123@users.noreply.github.com>

* fix: resolve PR comments

* fix: resolve PR comments

* fix: resolve PR comments

---------

Co-authored-by: shuiyisong <xixing.sys@gmail.com>
Co-authored-by: fys <40801205+Fengys123@users.noreply.github.com>
2023-03-24 04:28:34 +00:00
LFC
92963b9614 feat: execute "delete" in query engine (in the form of "LogicalPlan") (#1222)
fix: execute "delete" in query engine (in the form of "LogicalPlan")
2023-03-24 12:11:58 +08:00
Yingwen
f1139fba59 fix: Holds FileHandle in ParquetReader to avoid the purger purges it (#1224) 2023-03-23 14:24:25 +00:00
Ruihang Xia
4e552245b1 fix: range func tests (#1221)
* remove ignore on range fn tests

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

* placeholder for changes, deriv and resets

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

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2023-03-23 17:33:11 +08:00
Ruihang Xia
3126bbc1c7 docs: use CDN for logos (#1219)
Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2023-03-23 11:39:24 +08:00
LFC
b77b561bc8 refactor: execute insert with select in query engine (#1181)
* refactor: execute insert with select in query engine

* fix: resolve PR comments
2023-03-23 10:38:26 +08:00
dennis zhuang
501faad8ab chore: rename params in flush api (#1213) 2023-03-22 14:07:23 +08:00
Eugene Tolbakov
5397a9bbe6 feat(to_unixtime): add initial implementation (#1186)
* feat(to_unixtime): add initial implementation

* feat(to_unixtime): use Timestamp for conversion

* feat(to_unixtime):  implement conversion to Result<VectorRef>

* feat(to_unixtime): make unit test pass

* feat(to_unixtime): preserve None for invalid timestamps

* feat(to_unixtime): address code review suggestions

* feat(to_unixtime): add an sqlness test

* feat(to_unixtime): adjust the assertion for the sqlness test

* Update tests/cases/standalone/common/select/dummy.sql

---------

Co-authored-by: Ruihang Xia <waynestxia@gmail.com>
2023-03-21 12:41:07 +00:00
Ruihang Xia
f351ee7042 docs: update document string and site (#1211)
Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2023-03-21 07:01:08 +00:00
Ruihang Xia
e0493e0b8f feat: flush all tables on shutdown (#1185)
* feat: impl flush on shutdown (#14)

* feat: impl flush on shutdown

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

* powerful if-else!

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

---------

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

* retrieve table handler from schema provider

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

* feat: impl flush on shutdown

* feat: impl flush on shutdown

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

* powerful if-else!

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

---------

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

* retrieve table handler from schema provider

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

* Update src/datanode/src/instance.rs

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

* fix: uncommitted merge change

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

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
Co-authored-by: dennis zhuang <killme2008@gmail.com>
2023-03-21 14:36:30 +08:00
84 changed files with 1884 additions and 1345 deletions

1
Cargo.lock generated
View File

@@ -2294,6 +2294,7 @@ dependencies = [
"common-catalog",
"common-datasource",
"common-error",
"common-function",
"common-grpc",
"common-grpc-expr",
"common-procedure",

View File

@@ -1,8 +1,8 @@
<p align="center">
<picture>
<source media="(prefers-color-scheme: light)" srcset="/docs/logo-text-padding.png">
<source media="(prefers-color-scheme: dark)" srcset="/docs/logo-text-padding-dark.png">
<img alt="GreptimeDB Logo" src="/docs/logo-text-padding.png" width="400px">
<source media="(prefers-color-scheme: light)" srcset="https://cdn.jsdelivr.net/gh/GreptimeTeam/greptimedb@develop/docs/logo-text-padding.png">
<source media="(prefers-color-scheme: dark)" srcset="https://cdn.jsdelivr.net/gh/GreptimeTeam/greptimedb@develop/docs/logo-text-padding-dark.png">
<img alt="GreptimeDB Logo" src="https://cdn.jsdelivr.net/gh/GreptimeTeam/greptimedb@develop/docs/logo-text-padding.png" width="400px">
</picture>
</p>
@@ -158,6 +158,7 @@ You can always cleanup test database by removing `/tmp/greptimedb`.
- GreptimeDB [User Guide](https://docs.greptime.com/user-guide/concepts.html)
- GreptimeDB [Developer
Guide](https://docs.greptime.com/developer-guide/overview.html)
- GreptimeDB [internal code document](https://greptimedb.rs)
### Dashboard
- [The dashboard UI for GreptimeDB](https://github.com/GreptimeTeam/dashboard)

View File

@@ -20,7 +20,7 @@ use std::sync::Arc;
use api::v1::meta::{RegionStat, TableName};
use common_telemetry::{info, warn};
use snafu::{OptionExt, ResultExt};
use snafu::ResultExt;
use table::engine::{EngineContext, TableEngineRef};
use table::metadata::TableId;
use table::requests::CreateTableRequest;
@@ -228,34 +228,25 @@ pub(crate) async fn handle_system_table_request<'a, M: CatalogManager>(
/// The stat of regions in the datanode node.
/// The number of regions can be got from len of vec.
pub async fn datanode_stat(catalog_manager: &CatalogManagerRef) -> Result<(u64, Vec<RegionStat>)> {
///
/// Ignores any errors occurred during iterating regions. The intention of this method is to
/// collect region stats that will be carried in Datanode's heartbeat to Metasrv, so it's a
/// "try our best" job.
pub async fn datanode_stat(catalog_manager: &CatalogManagerRef) -> (u64, Vec<RegionStat>) {
let mut region_number: u64 = 0;
let mut region_stats = Vec::new();
for catalog_name in catalog_manager.catalog_names()? {
let catalog =
catalog_manager
.catalog(&catalog_name)?
.context(error::CatalogNotFoundSnafu {
catalog_name: &catalog_name,
})?;
let Ok(catalog_names) = catalog_manager.catalog_names() else { return (region_number, region_stats) };
for catalog_name in catalog_names {
let Ok(Some(catalog)) = catalog_manager.catalog(&catalog_name) else { continue };
for schema_name in catalog.schema_names()? {
let schema = catalog
.schema(&schema_name)?
.context(error::SchemaNotFoundSnafu {
catalog: &catalog_name,
schema: &schema_name,
})?;
let Ok(schema_names) = catalog.schema_names() else { continue };
for schema_name in schema_names {
let Ok(Some(schema)) = catalog.schema(&schema_name) else { continue };
for table_name in schema.table_names()? {
let table =
schema
.table(&table_name)
.await?
.context(error::TableNotFoundSnafu {
table_info: &table_name,
})?;
let Ok(table_names) = schema.table_names() else { continue };
for table_name in table_names {
let Ok(Some(table)) = schema.table(&table_name).await else { continue };
let region_numbers = &table.table_info().meta.region_numbers;
region_number += region_numbers.len() as u64;
@@ -282,6 +273,5 @@ pub async fn datanode_stat(catalog_manager: &CatalogManagerRef) -> Result<(u64,
}
}
}
Ok((region_number, region_stats))
(region_number, region_stats)
}

View File

@@ -12,6 +12,8 @@
// See the License for the specific language governing permissions and
// limitations under the License.
#![doc = include_str!("../../../../README.md")]
use std::fmt;
use clap::Parser;

View File

@@ -11,11 +11,17 @@
// 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;
mod to_unixtime;
use to_unixtime::ToUnixtimeFunction;
use crate::scalars::function_registry::FunctionRegistry;
pub(crate) struct TimestampFunction;
impl TimestampFunction {
pub fn register(_registry: &FunctionRegistry) {}
pub fn register(registry: &FunctionRegistry) {
registry.register(Arc::new(ToUnixtimeFunction::default()));
}
}

View File

@@ -0,0 +1,148 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use std::fmt;
use std::str::FromStr;
use std::sync::Arc;
use common_query::error::{self, Result, UnsupportedInputDataTypeSnafu};
use common_query::prelude::{Signature, Volatility};
use common_time::timestamp::TimeUnit;
use common_time::Timestamp;
use datatypes::prelude::ConcreteDataType;
use datatypes::types::StringType;
use datatypes::vectors::{Int64Vector, StringVector, Vector, VectorRef};
use snafu::ensure;
use crate::scalars::function::{Function, FunctionContext};
#[derive(Clone, Debug, Default)]
pub struct ToUnixtimeFunction;
const NAME: &str = "to_unixtime";
fn convert_to_seconds(arg: &str) -> Option<i64> {
match Timestamp::from_str(arg) {
Ok(ts) => {
let sec_mul = (TimeUnit::Second.factor() / ts.unit().factor()) as i64;
Some(ts.value().div_euclid(sec_mul))
}
Err(_err) => None,
}
}
impl Function for ToUnixtimeFunction {
fn name(&self) -> &str {
NAME
}
fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result<ConcreteDataType> {
Ok(ConcreteDataType::timestamp_second_datatype())
}
fn signature(&self) -> Signature {
Signature::exact(
vec![ConcreteDataType::String(StringType)],
Volatility::Immutable,
)
}
fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result<VectorRef> {
ensure!(
columns.len() == 1,
error::InvalidFuncArgsSnafu {
err_msg: format!(
"The length of the args is not correct, expect exactly one, have: {}",
columns.len()
),
}
);
match columns[0].data_type() {
ConcreteDataType::String(_) => {
let array = columns[0].to_arrow_array();
let vector = StringVector::try_from_arrow_array(&array).unwrap();
Ok(Arc::new(Int64Vector::from(
(0..vector.len())
.map(|i| convert_to_seconds(&vector.get(i).to_string()))
.collect::<Vec<_>>(),
)))
}
_ => UnsupportedInputDataTypeSnafu {
function: NAME,
datatypes: columns.iter().map(|c| c.data_type()).collect::<Vec<_>>(),
}
.fail(),
}
}
}
impl fmt::Display for ToUnixtimeFunction {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
write!(f, "TO_UNIXTIME")
}
}
#[cfg(test)]
mod tests {
use common_query::prelude::TypeSignature;
use datatypes::prelude::ConcreteDataType;
use datatypes::types::StringType;
use datatypes::value::Value;
use datatypes::vectors::StringVector;
use super::{ToUnixtimeFunction, *};
use crate::scalars::Function;
#[test]
fn test_to_unixtime() {
let f = ToUnixtimeFunction::default();
assert_eq!("to_unixtime", f.name());
assert_eq!(
ConcreteDataType::timestamp_second_datatype(),
f.return_type(&[]).unwrap()
);
assert!(matches!(f.signature(),
Signature {
type_signature: TypeSignature::Exact(valid_types),
volatility: Volatility::Immutable
} if valid_types == vec![ConcreteDataType::String(StringType)]
));
let times = vec![
Some("2023-03-01T06:35:02Z"),
None,
Some("2022-06-30T23:59:60Z"),
Some("invalid_time_stamp"),
];
let results = vec![Some(1677652502), None, Some(1656633600), None];
let args: Vec<VectorRef> = vec![Arc::new(StringVector::from(times.clone()))];
let vector = f.eval(FunctionContext::default(), &args).unwrap();
assert_eq!(4, vector.len());
for (i, _t) in times.iter().enumerate() {
let v = vector.get(i);
if i == 1 || i == 3 {
assert_eq!(Value::Null, v);
continue;
}
match v {
Value::Int64(ts) => {
assert_eq!(ts, (*results.get(i).unwrap()).unwrap());
}
_ => unreachable!(),
}
}
}
}

View File

@@ -17,6 +17,7 @@ use std::any::Any;
use common_error::ext::BoxedError;
use common_error::prelude::*;
use datatypes::prelude::ConcreteDataType;
pub type Result<T> = std::result::Result<T, Error>;
@@ -70,6 +71,26 @@ pub enum Error {
source: datafusion_common::DataFusionError,
backtrace: Backtrace,
},
#[snafu(display("Column {} not exists in table {}", column_name, table_name))]
ColumnNotExists {
column_name: String,
table_name: String,
backtrace: Backtrace,
},
#[snafu(display(
"Failed to cast vector of type '{:?}' to type '{:?}', source: {}",
from_type,
to_type,
source
))]
CastVector {
from_type: ConcreteDataType,
to_type: ConcreteDataType,
#[snafu(backtrace)]
source: datatypes::error::Error,
},
}
impl ErrorExt for Error {
@@ -81,11 +102,14 @@ impl ErrorExt for Error {
| Error::CreateRecordBatches { .. }
| Error::PollStream { .. }
| Error::Format { .. }
| Error::InitRecordbatchStream { .. } => StatusCode::Internal,
| Error::InitRecordbatchStream { .. }
| Error::ColumnNotExists { .. } => StatusCode::Internal,
Error::External { source } => source.status_code(),
Error::SchemaConversion { source, .. } => source.status_code(),
Error::SchemaConversion { source, .. } | Error::CastVector { source, .. } => {
source.status_code()
}
}
}

View File

@@ -12,14 +12,16 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::collections::HashMap;
use datatypes::schema::SchemaRef;
use datatypes::value::Value;
use datatypes::vectors::{Helper, VectorRef};
use serde::ser::{Error, SerializeStruct};
use serde::{Serialize, Serializer};
use snafu::ResultExt;
use snafu::{OptionExt, ResultExt};
use crate::error::{self, Result};
use crate::error::{self, CastVectorSnafu, ColumnNotExistsSnafu, Result};
use crate::DfRecordBatch;
/// A two-dimensional batch of column-oriented data with a defined schema.
@@ -108,6 +110,41 @@ impl RecordBatch {
pub fn rows(&self) -> RecordBatchRowIterator<'_> {
RecordBatchRowIterator::new(self)
}
pub fn column_vectors(
&self,
table_name: &str,
table_schema: SchemaRef,
) -> Result<HashMap<String, VectorRef>> {
let mut vectors = HashMap::with_capacity(self.num_columns());
// column schemas in recordbatch must match its vectors, otherwise it's corrupted
for (vector_schema, vector) in self.schema.column_schemas().iter().zip(self.columns.iter())
{
let column_name = &vector_schema.name;
let column_schema =
table_schema
.column_schema_by_name(column_name)
.context(ColumnNotExistsSnafu {
table_name,
column_name,
})?;
let vector = if vector_schema.data_type != column_schema.data_type {
vector
.cast(&column_schema.data_type)
.with_context(|_| CastVectorSnafu {
from_type: vector.data_type(),
to_type: column_schema.data_type.clone(),
})?
} else {
vector.clone()
};
vectors.insert(column_name.clone(), vector);
}
Ok(vectors)
}
}
impl Serialize for RecordBatch {

View File

@@ -21,6 +21,7 @@ common-base = { path = "../common/base" }
common-catalog = { path = "../common/catalog" }
common-error = { path = "../common/error" }
common-datasource = { path = "../common/datasource" }
common-function = { path = "../common/function" }
common-grpc = { path = "../common/grpc" }
common-grpc-expr = { path = "../common/grpc-expr" }
common-procedure = { path = "../common/procedure" }

View File

@@ -17,9 +17,7 @@ use std::any::Any;
use common_datasource::error::Error as DataSourceError;
use common_error::prelude::*;
use common_procedure::ProcedureId;
use common_recordbatch::error::Error as RecordBatchError;
use datafusion::parquet;
use datatypes::prelude::ConcreteDataType;
use storage::error::Error as StorageError;
use table::error::Error as TableError;
use url::ParseError;
@@ -125,24 +123,6 @@ pub enum Error {
))]
ColumnValuesNumberMismatch { columns: usize, values: usize },
#[snafu(display(
"Column type mismatch, column: {}, expected type: {:?}, actual: {:?}",
column,
expected,
actual,
))]
ColumnTypeMismatch {
column: String,
expected: ConcreteDataType,
actual: ConcreteDataType,
},
#[snafu(display("Failed to collect record batch, source: {}", source))]
CollectRecords {
#[snafu(backtrace)]
source: RecordBatchError,
},
#[snafu(display("Failed to parse sql value, source: {}", source))]
ParseSqlValue {
#[snafu(backtrace)]
@@ -556,8 +536,6 @@ impl ErrorExt for Error {
Insert { source, .. } => source.status_code(),
Delete { source, .. } => source.status_code(),
CollectRecords { source, .. } => source.status_code(),
TableNotFound { .. } => StatusCode::TableNotFound,
ColumnNotFound { .. } => StatusCode::TableColumnNotFound,
@@ -570,7 +548,6 @@ impl ErrorExt for Error {
ConvertSchema { source, .. } | VectorComputation { source } => source.status_code(),
ColumnValuesNumberMismatch { .. }
| ColumnTypeMismatch { .. }
| InvalidSql { .. }
| InvalidUrl { .. }
| InvalidPath { .. }

View File

@@ -106,13 +106,7 @@ impl HeartbeatTask {
let mut tx = Self::create_streams(&meta_client, running.clone()).await?;
common_runtime::spawn_bg(async move {
while running.load(Ordering::Acquire) {
let (region_num, region_stats) = match datanode_stat(&catalog_manager_clone).await {
Ok(datanode_stat) => (datanode_stat.0 as i64, datanode_stat.1),
Err(e) => {
error!("failed to get region status, err: {e:?}");
(-1, vec![])
}
};
let (region_num, region_stats) = datanode_stat(&catalog_manager_clone).await;
let req = HeartbeatRequest {
peer: Some(Peer {
@@ -120,7 +114,7 @@ impl HeartbeatTask {
addr: addr.clone(),
}),
node_stat: Some(NodeStat {
region_num,
region_num: region_num as _,
..Default::default()
}),
region_stats,

View File

@@ -37,12 +37,14 @@ use object_store::services::{Fs as FsBuilder, Oss as OSSBuilder, S3 as S3Builder
use object_store::{util, ObjectStore, ObjectStoreBuilder};
use query::query_engine::{QueryEngineFactory, QueryEngineRef};
use servers::Mode;
use session::context::QueryContext;
use snafu::prelude::*;
use storage::compaction::{CompactionHandler, CompactionSchedulerRef, SimplePicker};
use storage::config::EngineConfig as StorageEngineConfig;
use storage::scheduler::{LocalScheduler, SchedulerConfig};
use storage::EngineImpl;
use store_api::logstore::LogStore;
use table::requests::FlushTableRequest;
use table::table::numbers::NumbersTable;
use table::table::TableIdProviderRef;
use table::Table;
@@ -56,7 +58,7 @@ use crate::error::{
};
use crate::heartbeat::HeartbeatTask;
use crate::script::ScriptExecutor;
use crate::sql::SqlHandler;
use crate::sql::{SqlHandler, SqlRequest};
mod grpc;
mod script;
@@ -202,7 +204,6 @@ impl Instance {
sql_handler: SqlHandler::new(
table_engine.clone(),
catalog_manager.clone(),
query_engine.clone(),
table_engine,
procedure_manager,
),
@@ -233,6 +234,8 @@ impl Instance {
.context(ShutdownInstanceSnafu)?;
}
self.flush_tables().await?;
self.sql_handler
.close()
.await
@@ -240,6 +243,43 @@ impl Instance {
.context(ShutdownInstanceSnafu)
}
pub async fn flush_tables(&self) -> Result<()> {
info!("going to flush all schemas");
let schema_list = self
.catalog_manager
.catalog(DEFAULT_CATALOG_NAME)
.map_err(BoxedError::new)
.context(ShutdownInstanceSnafu)?
.expect("Default schema not found")
.schema_names()
.map_err(BoxedError::new)
.context(ShutdownInstanceSnafu)?;
let flush_requests = schema_list
.into_iter()
.map(|schema_name| {
SqlRequest::FlushTable(FlushTableRequest {
catalog_name: DEFAULT_CATALOG_NAME.to_string(),
schema_name,
table_name: None,
region_number: None,
wait: Some(true),
})
})
.collect::<Vec<_>>();
let flush_result = futures::future::try_join_all(
flush_requests
.into_iter()
.map(|request| self.sql_handler.execute(request, QueryContext::arc())),
)
.await
.map_err(BoxedError::new)
.context(ShutdownInstanceSnafu);
info!("Flushed all tables result: {}", flush_result.is_ok());
flush_result?;
Ok(())
}
pub fn sql_handler(&self) -> &SqlHandler {
&self.sql_handler
}

View File

@@ -21,7 +21,7 @@ use common_query::Output;
use query::parser::{PromQuery, QueryLanguageParser, QueryStatement};
use query::plan::LogicalPlan;
use servers::query_handler::grpc::GrpcQueryHandler;
use session::context::QueryContextRef;
use session::context::{QueryContext, QueryContextRef};
use snafu::prelude::*;
use sql::statements::statement::Statement;
use substrait::{DFLogicalSubstraitConvertor, SubstraitPlan};
@@ -53,7 +53,7 @@ impl Instance {
.context(DecodeLogicalPlanSnafu)?;
self.query_engine
.execute(&LogicalPlan::DfPlan(logical_plan))
.execute(LogicalPlan::DfPlan(logical_plan), QueryContext::arc())
.await
.context(ExecuteLogicalPlanSnafu)
}
@@ -69,11 +69,11 @@ impl Instance {
let plan = self
.query_engine
.planner()
.plan(stmt, ctx)
.plan(stmt, ctx.clone())
.await
.context(PlanStatementSnafu)?;
self.query_engine
.execute(&plan)
.execute(plan, ctx)
.await
.context(ExecuteLogicalPlanSnafu)
}
@@ -175,7 +175,7 @@ mod test {
.plan(stmt, QueryContext::arc())
.await
.unwrap();
engine.execute(&plan).await.unwrap()
engine.execute(plan, QueryContext::arc()).await.unwrap()
}
#[tokio::test(flavor = "multi_thread")]

View File

@@ -19,7 +19,6 @@ use common_error::prelude::BoxedError;
use common_query::Output;
use common_telemetry::logging::info;
use common_telemetry::timer;
use futures::StreamExt;
use query::error::QueryExecutionSnafu;
use query::parser::{PromQuery, QueryLanguageParser, QueryStatement};
use query::query_engine::StatementHandler;
@@ -39,8 +38,7 @@ use crate::error::{
};
use crate::instance::Instance;
use crate::metric;
use crate::sql::insert::InsertRequests;
use crate::sql::SqlRequest;
use crate::sql::{SqlHandler, SqlRequest};
impl Instance {
pub async fn execute_stmt(
@@ -50,37 +48,10 @@ impl Instance {
) -> Result<Output> {
match stmt {
QueryStatement::Sql(Statement::Insert(insert)) => {
let requests = self
.sql_handler
.insert_to_requests(self.catalog_manager.clone(), *insert, query_ctx.clone())
.await?;
match requests {
InsertRequests::Request(request) => {
self.sql_handler.execute(request, query_ctx.clone()).await
}
InsertRequests::Stream(mut s) => {
let mut rows = 0;
while let Some(request) = s.next().await {
match self
.sql_handler
.execute(request?, query_ctx.clone())
.await?
{
Output::AffectedRows(n) => {
rows += n;
}
_ => unreachable!(),
}
}
Ok(Output::AffectedRows(rows))
}
}
}
QueryStatement::Sql(Statement::Delete(delete)) => {
let request = SqlRequest::Delete(*delete);
self.sql_handler.execute(request, query_ctx).await
let request =
SqlHandler::insert_to_request(self.catalog_manager.clone(), *insert, query_ctx)
.await?;
self.sql_handler.insert(request).await
}
QueryStatement::Sql(Statement::CreateDatabase(create_database)) => {
let request = CreateDatabaseRequest {
@@ -210,6 +181,7 @@ impl Instance {
| QueryStatement::Sql(Statement::Explain(_))
| QueryStatement::Sql(Statement::Use(_))
| QueryStatement::Sql(Statement::Tql(_))
| QueryStatement::Sql(Statement::Delete(_))
| QueryStatement::Promql(_) => unreachable!(),
}
}
@@ -226,10 +198,13 @@ impl Instance {
let engine = self.query_engine();
let plan = engine
.planner()
.plan(stmt, query_ctx)
.plan(stmt, query_ctx.clone())
.await
.context(PlanStatementSnafu)?;
engine.execute(&plan).await.context(ExecuteStatementSnafu)
engine
.execute(plan, query_ctx)
.await
.context(ExecuteStatementSnafu)
}
// TODO(ruihang): merge this and `execute_promql` after #951 landed
@@ -262,10 +237,13 @@ impl Instance {
let engine = self.query_engine();
let plan = engine
.planner()
.plan(stmt, query_ctx)
.plan(stmt, query_ctx.clone())
.await
.context(PlanStatementSnafu)?;
engine.execute(&plan).await.context(ExecuteStatementSnafu)
engine
.execute(plan, query_ctx)
.await
.context(ExecuteStatementSnafu)
}
}

View File

@@ -17,11 +17,9 @@ use common_error::prelude::BoxedError;
use common_procedure::ProcedureManagerRef;
use common_query::Output;
use common_telemetry::error;
use query::query_engine::QueryEngineRef;
use query::sql::{describe_table, show_databases, show_tables};
use session::context::QueryContextRef;
use snafu::{OptionExt, ResultExt};
use sql::statements::delete::Delete;
use sql::statements::describe::DescribeTable;
use sql::statements::show::{ShowDatabases, ShowTables};
use table::engine::{EngineContext, TableEngineProcedureRef, TableEngineRef, TableReference};
@@ -37,14 +35,12 @@ mod alter;
mod copy_table_from;
mod copy_table_to;
mod create;
mod delete;
mod drop_table;
mod flush_table;
pub(crate) mod insert;
#[derive(Debug)]
pub enum SqlRequest {
Insert(InsertRequest),
CreateTable(CreateTableRequest),
CreateDatabase(CreateDatabaseRequest),
Alter(AlterTableRequest),
@@ -53,15 +49,14 @@ pub enum SqlRequest {
ShowDatabases(ShowDatabases),
ShowTables(ShowTables),
DescribeTable(DescribeTable),
Delete(Delete),
CopyTable(CopyTableRequest),
}
// Handler to execute SQL except query
#[derive(Clone)]
pub struct SqlHandler {
table_engine: TableEngineRef,
catalog_manager: CatalogManagerRef,
query_engine: QueryEngineRef,
engine_procedure: TableEngineProcedureRef,
procedure_manager: Option<ProcedureManagerRef>,
}
@@ -70,14 +65,12 @@ impl SqlHandler {
pub fn new(
table_engine: TableEngineRef,
catalog_manager: CatalogManagerRef,
query_engine: QueryEngineRef,
engine_procedure: TableEngineProcedureRef,
procedure_manager: Option<ProcedureManagerRef>,
) -> Self {
Self {
table_engine,
catalog_manager,
query_engine,
engine_procedure,
procedure_manager,
}
@@ -89,12 +82,10 @@ impl SqlHandler {
// there, instead of executing here in a "static" fashion.
pub async fn execute(&self, request: SqlRequest, query_ctx: QueryContextRef) -> Result<Output> {
let result = match request {
SqlRequest::Insert(req) => self.insert(req).await,
SqlRequest::CreateTable(req) => self.create_table(req).await,
SqlRequest::CreateDatabase(req) => self.create_database(req, query_ctx.clone()).await,
SqlRequest::Alter(req) => self.alter(req).await,
SqlRequest::DropTable(req) => self.drop_table(req).await,
SqlRequest::Delete(req) => self.delete(query_ctx.clone(), req).await,
SqlRequest::CopyTable(req) => match req.direction {
CopyDirection::Export => self.copy_table_to(req).await,
CopyDirection::Import => self.copy_table_from(req).await,
@@ -150,239 +141,3 @@ impl SqlHandler {
.context(CloseTableEngineSnafu)
}
}
#[cfg(test)]
mod tests {
use std::any::Any;
use std::sync::Arc;
use catalog::{CatalogManager, RegisterTableRequest};
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
use common_query::logical_plan::Expr;
use common_query::physical_plan::PhysicalPlanRef;
use common_test_util::temp_dir::create_temp_dir;
use common_time::timestamp::Timestamp;
use datatypes::prelude::ConcreteDataType;
use datatypes::schema::{ColumnSchema, SchemaBuilder, SchemaRef};
use datatypes::value::Value;
use futures::StreamExt;
use log_store::NoopLogStore;
use mito::config::EngineConfig as TableEngineConfig;
use mito::engine::MitoEngine;
use object_store::services::Fs as Builder;
use object_store::{ObjectStore, ObjectStoreBuilder};
use query::parser::{QueryLanguageParser, QueryStatement};
use query::QueryEngineFactory;
use session::context::QueryContext;
use sql::statements::statement::Statement;
use storage::compaction::noop::NoopCompactionScheduler;
use storage::config::EngineConfig as StorageEngineConfig;
use storage::EngineImpl;
use table::error::Result as TableResult;
use table::metadata::TableInfoRef;
use table::Table;
use super::*;
use crate::error::Error;
use crate::sql::insert::InsertRequests;
struct DemoTable;
#[async_trait::async_trait]
impl Table for DemoTable {
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
let column_schemas = vec![
ColumnSchema::new("host", ConcreteDataType::string_datatype(), false),
ColumnSchema::new("cpu", ConcreteDataType::float64_datatype(), true),
ColumnSchema::new("memory", ConcreteDataType::float64_datatype(), true),
ColumnSchema::new(
"ts",
ConcreteDataType::timestamp_millisecond_datatype(),
true,
)
.with_time_index(true),
];
Arc::new(
SchemaBuilder::try_from(column_schemas)
.unwrap()
.build()
.unwrap(),
)
}
fn table_info(&self) -> TableInfoRef {
unimplemented!()
}
async fn scan(
&self,
_projection: Option<&Vec<usize>>,
_filters: &[Expr],
_limit: Option<usize>,
) -> TableResult<PhysicalPlanRef> {
unimplemented!();
}
}
#[tokio::test]
async fn test_statement_to_request() {
let dir = create_temp_dir("setup_test_engine_and_table");
let store_dir = dir.path().to_string_lossy();
let accessor = Builder::default().root(&store_dir).build().unwrap();
let object_store = ObjectStore::new(accessor).finish();
let compaction_scheduler = Arc::new(NoopCompactionScheduler::default());
let sql = r#"insert into demo(host, cpu, memory, ts) values
('host1', 66.6, 1024, 1655276557000),
('host2', 88.8, 333.3, 1655276558000)
"#;
let table_engine = Arc::new(MitoEngine::<EngineImpl<NoopLogStore>>::new(
TableEngineConfig::default(),
EngineImpl::new(
StorageEngineConfig::default(),
Arc::new(NoopLogStore::default()),
object_store.clone(),
compaction_scheduler,
),
object_store,
));
let catalog_list = Arc::new(
catalog::local::LocalCatalogManager::try_new(table_engine.clone())
.await
.unwrap(),
);
catalog_list.start().await.unwrap();
assert!(catalog_list
.register_table(RegisterTableRequest {
catalog: DEFAULT_CATALOG_NAME.to_string(),
schema: DEFAULT_SCHEMA_NAME.to_string(),
table_name: "demo".to_string(),
table_id: 1,
table: Arc::new(DemoTable),
})
.await
.unwrap());
let factory = QueryEngineFactory::new(catalog_list.clone());
let query_engine = factory.query_engine();
let sql_handler = SqlHandler::new(
table_engine.clone(),
catalog_list.clone(),
query_engine.clone(),
table_engine,
None,
);
let stmt = match QueryLanguageParser::parse_sql(sql).unwrap() {
QueryStatement::Sql(Statement::Insert(i)) => i,
_ => {
unreachable!()
}
};
let request = sql_handler
.insert_to_requests(catalog_list.clone(), *stmt, QueryContext::arc())
.await
.unwrap();
match request {
InsertRequests::Request(SqlRequest::Insert(req)) => {
assert_eq!(req.table_name, "demo");
let columns_values = req.columns_values;
assert_eq!(4, columns_values.len());
let hosts = &columns_values["host"];
assert_eq!(2, hosts.len());
assert_eq!(Value::from("host1"), hosts.get(0));
assert_eq!(Value::from("host2"), hosts.get(1));
let cpus = &columns_values["cpu"];
assert_eq!(2, cpus.len());
assert_eq!(Value::from(66.6f64), cpus.get(0));
assert_eq!(Value::from(88.8f64), cpus.get(1));
let memories = &columns_values["memory"];
assert_eq!(2, memories.len());
assert_eq!(Value::from(1024f64), memories.get(0));
assert_eq!(Value::from(333.3f64), memories.get(1));
let ts = &columns_values["ts"];
assert_eq!(2, ts.len());
assert_eq!(
Value::from(Timestamp::new_millisecond(1655276557000i64)),
ts.get(0)
);
assert_eq!(
Value::from(Timestamp::new_millisecond(1655276558000i64)),
ts.get(1)
);
}
_ => {
panic!("Not supposed to reach here")
}
}
// test inert into select
// type mismatch
let sql = "insert into demo(ts) select number from numbers limit 3";
let stmt = match QueryLanguageParser::parse_sql(sql).unwrap() {
QueryStatement::Sql(Statement::Insert(i)) => i,
_ => {
unreachable!()
}
};
let request = sql_handler
.insert_to_requests(catalog_list.clone(), *stmt, QueryContext::arc())
.await
.unwrap();
match request {
InsertRequests::Stream(mut stream) => {
assert!(matches!(
stream.next().await.unwrap().unwrap_err(),
Error::ColumnTypeMismatch { .. }
));
}
_ => unreachable!(),
}
let sql = "insert into demo(cpu) select cast(number as double) from numbers limit 3";
let stmt = match QueryLanguageParser::parse_sql(sql).unwrap() {
QueryStatement::Sql(Statement::Insert(i)) => i,
_ => {
unreachable!()
}
};
let request = sql_handler
.insert_to_requests(catalog_list.clone(), *stmt, QueryContext::arc())
.await
.unwrap();
match request {
InsertRequests::Stream(mut stream) => {
let mut times = 0;
while let Some(Ok(SqlRequest::Insert(req))) = stream.next().await {
times += 1;
assert_eq!(req.table_name, "demo");
let columns_values = req.columns_values;
assert_eq!(1, columns_values.len());
let memories = &columns_values["cpu"];
assert_eq!(3, memories.len());
assert_eq!(Value::from(0.0f64), memories.get(0));
assert_eq!(Value::from(1.0f64), memories.get(1));
assert_eq!(Value::from(2.0f64), memories.get(2));
}
assert_eq!(1, times);
}
_ => unreachable!(),
}
}
}

View File

@@ -1,142 +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 std::sync::Arc;
use common_query::Output;
use datatypes::data_type::DataType;
use datatypes::prelude::VectorRef;
use datatypes::vectors::StringVector;
use session::context::QueryContextRef;
use snafu::{OptionExt, ResultExt};
use sql::ast::{BinaryOperator, Expr, Value};
use sql::statements::delete::Delete;
use sql::statements::sql_value_to_value;
use table::engine::TableReference;
use table::requests::DeleteRequest;
use table::TableRef;
use crate::error::{ColumnNotFoundSnafu, DeleteSnafu, InvalidSqlSnafu, NotSupportSqlSnafu, Result};
use crate::instance::sql::table_idents_to_full_name;
use crate::sql::SqlHandler;
impl SqlHandler {
pub(crate) async fn delete(&self, query_ctx: QueryContextRef, stmt: Delete) -> Result<Output> {
let (catalog_name, schema_name, table_name) =
table_idents_to_full_name(stmt.table_name(), query_ctx)?;
let table_ref = TableReference {
catalog: &catalog_name.to_string(),
schema: &schema_name.to_string(),
table: &table_name.to_string(),
};
let table = self.get_table(&table_ref)?;
let req = DeleteRequest {
key_column_values: parse_selection(stmt.selection(), &table)?,
};
let affected_rows = table.delete(req).await.with_context(|_| DeleteSnafu {
table_name: table_ref.to_string(),
})?;
Ok(Output::AffectedRows(affected_rows))
}
}
/// parse selection, currently supported format is `tagkey1 = 'tagvalue1' and 'ts' = 'value'`.
/// (only uses =, and in the where clause and provides all columns needed by the key.)
fn parse_selection(
selection: &Option<Expr>,
table: &TableRef,
) -> Result<HashMap<String, VectorRef>> {
let mut key_column_values = HashMap::new();
if let Some(expr) = selection {
parse_expr(expr, &mut key_column_values, table)?;
}
Ok(key_column_values)
}
fn parse_expr(
expr: &Expr,
key_column_values: &mut HashMap<String, VectorRef>,
table: &TableRef,
) -> Result<()> {
// match BinaryOp
if let Expr::BinaryOp { left, op, right } = expr {
match (&**left, op, &**right) {
// match And operator
(Expr::BinaryOp { .. }, BinaryOperator::And, Expr::BinaryOp { .. }) => {
parse_expr(left, key_column_values, table)?;
parse_expr(right, key_column_values, table)?;
return Ok(());
}
// match Eq operator
(Expr::Identifier(column_name), BinaryOperator::Eq, Expr::Value(value)) => {
key_column_values.insert(
column_name.to_string(),
value_to_vector(&column_name.to_string(), value, table)?,
);
return Ok(());
}
(Expr::Identifier(column_name), BinaryOperator::Eq, Expr::Identifier(value)) => {
key_column_values.insert(
column_name.to_string(),
Arc::new(StringVector::from(vec![value.to_string()])),
);
return Ok(());
}
_ => {}
}
}
NotSupportSqlSnafu {
msg: format!(
"Not support sql expr:{expr},correct format is tagkey1 = tagvalue1 and ts = value"
),
}
.fail()
}
/// parse value to vector
fn value_to_vector(column_name: &String, sql_value: &Value, table: &TableRef) -> Result<VectorRef> {
let schema = table.schema();
let column_schema =
schema
.column_schema_by_name(column_name)
.with_context(|| ColumnNotFoundSnafu {
table_name: table.table_info().name.clone(),
column_name: column_name.to_string(),
})?;
let data_type = &column_schema.data_type;
let value = sql_value_to_value(column_name, data_type, sql_value);
match value {
Ok(value) => {
let mut vec = data_type.create_mutable_vector(1);
if vec.try_push_value_ref(value.as_value_ref()).is_err() {
return InvalidSqlSnafu {
msg: format!(
"invalid sql, column name is {column_name}, value is {sql_value}",
),
}
.fail();
}
Ok(vec.to_vector())
}
_ => InvalidSqlSnafu {
msg: format!("invalid sql, column name is {column_name}, value is {sql_value}",),
}
.fail(),
}
}

View File

@@ -12,9 +12,9 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use catalog::SchemaProviderRef;
use common_query::Output;
use snafu::{OptionExt, ResultExt};
use table::engine::TableReference;
use table::requests::FlushTableRequest;
use crate::error::{self, CatalogSnafu, DatabaseNotFoundSnafu, Result};
@@ -22,34 +22,22 @@ use crate::sql::SqlHandler;
impl SqlHandler {
pub(crate) async fn flush_table(&self, req: FlushTableRequest) -> Result<Output> {
if let Some(table) = &req.table_name {
self.flush_table_inner(
&req.catalog_name,
&req.schema_name,
table,
req.region_number,
req.wait,
)
.await?;
} else {
let schema = self
.catalog_manager
.schema(&req.catalog_name, &req.schema_name)
.context(CatalogSnafu)?
.context(DatabaseNotFoundSnafu {
catalog: &req.catalog_name,
schema: &req.schema_name,
})?;
let schema = self
.catalog_manager
.schema(&req.catalog_name, &req.schema_name)
.context(CatalogSnafu)?
.context(DatabaseNotFoundSnafu {
catalog: &req.catalog_name,
schema: &req.schema_name,
})?;
if let Some(table) = &req.table_name {
self.flush_table_inner(schema, table, req.region_number, req.wait)
.await?;
} else {
let all_table_names = schema.table_names().context(CatalogSnafu)?;
futures::future::join_all(all_table_names.iter().map(|table| {
self.flush_table_inner(
&req.catalog_name,
&req.schema_name,
table,
req.region_number,
req.wait,
)
self.flush_table_inner(schema.clone(), table, req.region_number, req.wait)
}))
.await
.into_iter()
@@ -60,25 +48,18 @@ impl SqlHandler {
async fn flush_table_inner(
&self,
catalog: &str,
schema: &str,
table: &str,
schema: SchemaProviderRef,
table_name: &str,
region: Option<u32>,
wait: Option<bool>,
) -> Result<()> {
let table_ref = TableReference {
catalog,
schema,
table,
};
let full_table_name = table_ref.to_string();
let table = self.get_table(&table_ref)?;
table
schema
.table(table_name)
.await
.context(error::FindTableSnafu { table_name })?
.context(error::TableNotFoundSnafu { table_name })?
.flush(region, wait)
.await
.context(error::FlushTableSnafu {
table_name: full_table_name,
})
.context(error::FlushTableSnafu { table_name })
}
}

View File

@@ -11,49 +11,31 @@
// 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::pin::Pin;
use catalog::CatalogManagerRef;
use common_catalog::format_full_table_name;
use common_query::Output;
use common_recordbatch::RecordBatch;
use datafusion_expr::type_coercion::binary::coerce_types;
use datafusion_expr::Operator;
use datatypes::data_type::DataType;
use datatypes::schema::ColumnSchema;
use datatypes::vectors::MutableVector;
use futures::stream::{self, StreamExt};
use futures::Stream;
use query::parser::QueryStatement;
use session::context::QueryContextRef;
use snafu::{ensure, OptionExt, ResultExt};
use sql::ast::Value as SqlValue;
use sql::statements::insert::Insert;
use sql::statements::statement::Statement;
use sql::statements::{self};
use table::engine::TableReference;
use table::requests::*;
use table::TableRef;
use crate::error::{
CatalogSnafu, CollectRecordsSnafu, ColumnDefaultValueSnafu, ColumnNoneDefaultValueSnafu,
ColumnNotFoundSnafu, ColumnTypeMismatchSnafu, ColumnValuesNumberMismatchSnafu, Error,
ExecuteLogicalPlanSnafu, InsertSnafu, MissingInsertBodySnafu, ParseSqlSnafu,
ParseSqlValueSnafu, PlanStatementSnafu, Result, TableNotFoundSnafu,
CatalogSnafu, ColumnDefaultValueSnafu, ColumnNoneDefaultValueSnafu, ColumnNotFoundSnafu,
ColumnValuesNumberMismatchSnafu, InsertSnafu, MissingInsertBodySnafu, ParseSqlSnafu,
ParseSqlValueSnafu, Result, TableNotFoundSnafu,
};
use crate::sql::{table_idents_to_full_name, SqlHandler, SqlRequest};
use crate::sql::{table_idents_to_full_name, SqlHandler};
const DEFAULT_PLACEHOLDER_VALUE: &str = "default";
type InsertRequestStream = Pin<Box<dyn Stream<Item = Result<SqlRequest>> + Send>>;
pub(crate) enum InsertRequests {
// Single request
Request(SqlRequest),
// Streaming requests
Stream(InsertRequestStream),
}
impl SqlHandler {
pub(crate) async fn insert(&self, req: InsertRequest) -> Result<Output> {
// FIXME(dennis): table_ref is used in InsertSnafu and the req is consumed
@@ -77,7 +59,7 @@ impl SqlHandler {
table_ref: TableReference,
table: &TableRef,
stmt: Insert,
) -> Result<SqlRequest> {
) -> Result<InsertRequest> {
let values = stmt
.values_body()
.context(ParseSqlValueSnafu)?
@@ -129,7 +111,7 @@ impl SqlHandler {
}
}
Ok(SqlRequest::Insert(InsertRequest {
Ok(InsertRequest {
catalog_name: table_ref.catalog.to_string(),
schema_name: table_ref.schema.to_string(),
table_name: table_ref.table.to_string(),
@@ -138,150 +120,14 @@ impl SqlHandler {
.map(|(cs, mut b)| (cs.name.to_string(), b.to_vector()))
.collect(),
region_number: 0,
}))
})
}
fn build_request_from_batch(
stmt: Insert,
table: TableRef,
batch: RecordBatch,
query_ctx: QueryContextRef,
) -> Result<SqlRequest> {
let (catalog_name, schema_name, table_name) =
table_idents_to_full_name(stmt.table_name(), query_ctx)?;
let schema = table.schema();
let columns: Vec<_> = if stmt.columns().is_empty() {
schema
.column_schemas()
.iter()
.map(|c| c.name.to_string())
.collect()
} else {
stmt.columns().iter().map(|c| (*c).clone()).collect()
};
let columns_num = columns.len();
ensure!(
batch.num_columns() == columns_num,
ColumnValuesNumberMismatchSnafu {
columns: columns_num,
values: batch.num_columns(),
}
);
let batch_schema = &batch.schema;
let batch_columns = batch_schema.column_schemas();
assert_eq!(batch_columns.len(), columns_num);
let mut columns_values = HashMap::with_capacity(columns_num);
for (i, column_name) in columns.into_iter().enumerate() {
let column_schema = schema
.column_schema_by_name(&column_name)
.with_context(|| ColumnNotFoundSnafu {
table_name: &table_name,
column_name: &column_name,
})?;
let expect_datatype = column_schema.data_type.as_arrow_type();
// It's safe to retrieve the column schema by index, we already
// check columns number is the same above.
let batch_datatype = batch_columns[i].data_type.as_arrow_type();
let coerced_type = coerce_types(&expect_datatype, &Operator::Eq, &batch_datatype)
.map_err(|_| Error::ColumnTypeMismatch {
column: column_name.clone(),
expected: column_schema.data_type.clone(),
actual: batch_columns[i].data_type.clone(),
})?;
ensure!(
expect_datatype == coerced_type,
ColumnTypeMismatchSnafu {
column: column_name,
expected: column_schema.data_type.clone(),
actual: batch_columns[i].data_type.clone(),
}
);
let vector = batch
.column(i)
.cast(&column_schema.data_type)
.map_err(|_| Error::ColumnTypeMismatch {
column: column_name.clone(),
expected: column_schema.data_type.clone(),
actual: batch_columns[i].data_type.clone(),
})?;
columns_values.insert(column_name, vector);
}
Ok(SqlRequest::Insert(InsertRequest {
catalog_name,
schema_name,
table_name,
columns_values,
region_number: 0,
}))
}
// FIXME(dennis): move it to frontend when refactor is done.
async fn build_stream_from_query(
&self,
table: TableRef,
stmt: Insert,
query_ctx: QueryContextRef,
) -> Result<InsertRequestStream> {
let query = stmt
.query_body()
.context(ParseSqlValueSnafu)?
.context(MissingInsertBodySnafu)?;
let logical_plan = self
.query_engine
.planner()
.plan(
QueryStatement::Sql(Statement::Query(Box::new(query))),
query_ctx.clone(),
)
.await
.context(PlanStatementSnafu)?;
let output = self
.query_engine
.execute(&logical_plan)
.await
.context(ExecuteLogicalPlanSnafu)?;
let stream: InsertRequestStream = match output {
Output::RecordBatches(batches) => {
Box::pin(stream::iter(batches.take()).map(move |batch| {
Self::build_request_from_batch(
stmt.clone(),
table.clone(),
batch,
query_ctx.clone(),
)
}))
}
Output::Stream(stream) => Box::pin(stream.map(move |batch| {
Self::build_request_from_batch(
stmt.clone(),
table.clone(),
batch.context(CollectRecordsSnafu)?,
query_ctx.clone(),
)
})),
_ => unreachable!(),
};
Ok(stream)
}
pub(crate) async fn insert_to_requests(
&self,
pub async fn insert_to_request(
catalog_manager: CatalogManagerRef,
stmt: Insert,
query_ctx: QueryContextRef,
) -> Result<InsertRequests> {
) -> Result<InsertRequest> {
let (catalog_name, schema_name, table_name) =
table_idents_to_full_name(stmt.table_name(), query_ctx.clone())?;
@@ -293,16 +139,8 @@ impl SqlHandler {
table_name: format_full_table_name(&catalog_name, &schema_name, &table_name),
})?;
if stmt.is_insert_select() {
Ok(InsertRequests::Stream(
self.build_stream_from_query(table, stmt, query_ctx).await?,
))
} else {
let table_ref = TableReference::full(&catalog_name, &schema_name, &table_name);
Ok(InsertRequests::Request(Self::build_request_from_values(
table_ref, &table, stmt,
)?))
}
let table_ref = TableReference::full(&catalog_name, &schema_name, &table_name);
Self::build_request_from_values(table_ref, &table, stmt)
}
}

View File

@@ -22,7 +22,7 @@ use common_telemetry::logging;
use datatypes::data_type::ConcreteDataType;
use datatypes::vectors::{Int64Vector, StringVector, UInt64Vector, VectorRef};
use query::parser::{QueryLanguageParser, QueryStatement};
use session::context::QueryContext;
use session::context::{QueryContext, QueryContextRef};
use snafu::ResultExt;
use sql::statements::statement::Statement;
@@ -217,20 +217,20 @@ async fn test_execute_insert_by_select() {
try_execute_sql(&instance, "insert into demo2(host) select * from demo1")
.await
.unwrap_err(),
Error::ColumnValuesNumberMismatch { .. }
Error::PlanStatement { .. }
));
assert!(matches!(
try_execute_sql(&instance, "insert into demo2 select cpu,memory from demo1")
.await
.unwrap_err(),
Error::ColumnValuesNumberMismatch { .. }
Error::PlanStatement { .. }
));
assert!(matches!(
try_execute_sql(&instance, "insert into demo2(ts) select memory from demo1")
.await
.unwrap_err(),
Error::ColumnTypeMismatch { .. }
Error::PlanStatement { .. }
));
let output = execute_sql(&instance, "insert into demo2 select * from demo1").await;
@@ -751,7 +751,7 @@ async fn test_delete() {
let output = execute_sql(
&instance,
"delete from test_table where host = host1 and ts = 1655276557000 ",
"delete from test_table where host = 'host1' and ts = 1655276557000 ",
)
.await;
assert!(matches!(output, Output::AffectedRows(1)));
@@ -962,16 +962,30 @@ async fn try_execute_sql_in_db(
) -> Result<Output, crate::error::Error> {
let query_ctx = Arc::new(QueryContext::with(DEFAULT_CATALOG_NAME, db));
async fn plan_exec(
instance: &MockInstance,
stmt: QueryStatement,
query_ctx: QueryContextRef,
) -> Result<Output, Error> {
let engine = instance.inner().query_engine();
let plan = engine
.planner()
.plan(stmt, query_ctx.clone())
.await
.context(PlanStatementSnafu)?;
engine
.execute(plan, query_ctx)
.await
.context(ExecuteLogicalPlanSnafu)
}
let stmt = QueryLanguageParser::parse_sql(sql).unwrap();
match stmt {
QueryStatement::Sql(Statement::Query(_)) => {
let engine = instance.inner().query_engine();
let plan = engine
.planner()
.plan(stmt, query_ctx)
.await
.context(PlanStatementSnafu)?;
engine.execute(&plan).await.context(ExecuteLogicalPlanSnafu)
QueryStatement::Sql(Statement::Query(_)) | QueryStatement::Sql(Statement::Delete(_)) => {
plan_exec(instance, stmt, query_ctx).await
}
QueryStatement::Sql(Statement::Insert(ref insert)) if insert.is_insert_select() => {
plan_exec(instance, stmt, query_ctx).await
}
_ => instance.inner().execute_stmt(stmt, query_ctx).await,
}

View File

@@ -24,7 +24,6 @@ use datatypes::schema::{ColumnSchema, RawSchema};
use mito::config::EngineConfig;
use mito::table::test_util::{new_test_object_store, MockEngine, MockMitoEngine};
use query::parser::{PromQuery, QueryLanguageParser, QueryStatement};
use query::QueryEngineFactory;
use servers::Mode;
use session::context::QueryContext;
use snafu::ResultExt;
@@ -87,7 +86,7 @@ impl MockInstance {
match stmt {
QueryStatement::Sql(Statement::Query(_)) => {
let plan = planner.plan(stmt, QueryContext::arc()).await.unwrap();
engine.execute(&plan).await.unwrap()
engine.execute(plan, QueryContext::arc()).await.unwrap()
}
QueryStatement::Sql(Statement::Tql(tql)) => {
let plan = match tql {
@@ -103,7 +102,7 @@ impl MockInstance {
}
Tql::Explain(_) => unimplemented!(),
};
engine.execute(&plan).await.unwrap()
engine.execute(plan, QueryContext::arc()).await.unwrap()
}
_ => self
.inner()
@@ -201,17 +200,7 @@ pub async fn create_mock_sql_handler() -> SqlHandler {
.await
.unwrap(),
);
let catalog_list = catalog::local::new_memory_catalog_list().unwrap();
let factory = QueryEngineFactory::new(catalog_list);
SqlHandler::new(
mock_engine.clone(),
catalog_manager,
factory.query_engine(),
mock_engine,
None,
)
SqlHandler::new(mock_engine.clone(), catalog_manager, mock_engine, None)
}
pub(crate) async fn setup_test_instance(test_name: &str) -> MockInstance {

View File

@@ -428,52 +428,63 @@ fn parse_stmt(sql: &str) -> Result<Vec<Statement>> {
}
impl Instance {
async fn plan_exec(&self, stmt: Statement, query_ctx: QueryContextRef) -> Result<Output> {
let planner = self.query_engine.planner();
let plan = planner
.plan(QueryStatement::Sql(stmt), query_ctx.clone())
.await
.context(PlanStatementSnafu)?;
self.query_engine
.execute(plan, query_ctx)
.await
.context(ExecLogicalPlanSnafu)
}
async fn execute_tql(&self, tql: Tql, query_ctx: QueryContextRef) -> Result<Output> {
let plan = match tql {
Tql::Eval(eval) => {
let promql = PromQuery {
start: eval.start,
end: eval.end,
step: eval.step,
query: eval.query,
};
let stmt = QueryLanguageParser::parse_promql(&promql).context(ParseQuerySnafu)?;
self.query_engine
.planner()
.plan(stmt, query_ctx.clone())
.await
.context(PlanStatementSnafu)?
}
Tql::Explain(_) => unimplemented!(),
};
self.query_engine
.execute(plan, query_ctx)
.await
.context(ExecLogicalPlanSnafu)
}
async fn query_statement(&self, stmt: Statement, query_ctx: QueryContextRef) -> Result<Output> {
check_permission(self.plugins.clone(), &stmt, &query_ctx)?;
let planner = self.query_engine.planner();
match stmt {
Statement::Query(_) | Statement::Explain(_) => {
let plan = planner
.plan(QueryStatement::Sql(stmt), query_ctx)
.await
.context(PlanStatementSnafu)?;
self.query_engine
.execute(&plan)
.await
.context(ExecLogicalPlanSnafu)
Statement::Query(_) | Statement::Explain(_) | Statement::Delete(_) => {
self.plan_exec(stmt, query_ctx).await
}
Statement::Tql(tql) => {
let plan = match tql {
Tql::Eval(eval) => {
let promql = PromQuery {
start: eval.start,
end: eval.end,
step: eval.step,
query: eval.query,
};
let stmt =
QueryLanguageParser::parse_promql(&promql).context(ParseQuerySnafu)?;
planner
.plan(stmt, query_ctx)
.await
.context(PlanStatementSnafu)?
}
Tql::Explain(_) => unimplemented!(),
};
self.query_engine
.execute(&plan)
.await
.context(ExecLogicalPlanSnafu)
// For performance consideration, only "insert with select" is executed by query engine.
// Plain insert ("insert with values") is still executed directly in statement.
Statement::Insert(ref insert) if insert.is_insert_select() => {
self.plan_exec(stmt, query_ctx).await
}
Statement::Tql(tql) => self.execute_tql(tql, query_ctx).await,
Statement::CreateDatabase(_)
| Statement::ShowDatabases(_)
| Statement::CreateTable(_)
| Statement::ShowTables(_)
| Statement::DescribeTable(_)
| Statement::Insert(_)
| Statement::Delete(_)
| Statement::Alter(_)
| Statement::DropTable(_)
| Statement::Copy(_) => self
@@ -647,8 +658,8 @@ pub fn check_permission(
}
match stmt {
// query,explain and tql will be checked in QueryEngineState
Statement::Query(_) | Statement::Explain(_) | Statement::Tql(_) => {}
// These are executed by query engine, and will be checked there.
Statement::Query(_) | Statement::Explain(_) | Statement::Tql(_) | Statement::Delete(_) => {}
// database ops won't be checked
Statement::CreateDatabase(_) | Statement::ShowDatabases(_) | Statement::Use(_) => {}
// show create table and alter are not supported yet
@@ -673,9 +684,6 @@ pub fn check_permission(
Statement::DescribeTable(stmt) => {
validate_param(stmt.name(), query_ctx)?;
}
Statement::Delete(delete) => {
validate_param(delete.table_name(), query_ctx)?;
}
Statement::Copy(stmd) => match stmd {
CopyTable::To(copy_table_to) => validate_param(&copy_table_to.table_name, query_ctx)?,
CopyTable::From(copy_table_from) => {
@@ -1086,7 +1094,7 @@ mod tests {
.plan(stmt.clone(), QueryContext::arc())
.await
.unwrap();
let output = engine.execute(&plan).await.unwrap();
let output = engine.execute(plan, QueryContext::arc()).await.unwrap();
let Output::Stream(stream) = output else { unreachable!() };
let recordbatches = RecordBatches::try_collect(stream).await.unwrap();
let actual = recordbatches.pretty_print().unwrap();

View File

@@ -33,6 +33,7 @@ use common_error::prelude::BoxedError;
use common_query::Output;
use common_telemetry::{debug, info};
use datanode::instance::sql::table_idents_to_full_name;
use datanode::sql::SqlHandler;
use datatypes::prelude::ConcreteDataType;
use datatypes::schema::RawSchema;
use meta_client::client::MetaClient;
@@ -60,13 +61,12 @@ use crate::catalog::FrontendCatalogManager;
use crate::datanode::DatanodeClients;
use crate::error::{
self, AlterExprToRequestSnafu, CatalogEntrySerdeSnafu, CatalogSnafu, ColumnDataTypeSnafu,
DeserializePartitionSnafu, NotSupportedSnafu, ParseSqlSnafu, PrimaryKeyNotFoundSnafu,
RequestDatanodeSnafu, RequestMetaSnafu, Result, SchemaExistsSnafu, StartMetaClientSnafu,
TableAlreadyExistSnafu, TableNotFoundSnafu, TableSnafu, ToTableInsertRequestSnafu,
UnrecognizedTableOptionSnafu,
DeserializePartitionSnafu, InvokeDatanodeSnafu, NotSupportedSnafu, ParseSqlSnafu,
PrimaryKeyNotFoundSnafu, RequestDatanodeSnafu, RequestMetaSnafu, Result, SchemaExistsSnafu,
StartMetaClientSnafu, TableAlreadyExistSnafu, TableNotFoundSnafu, TableSnafu,
ToTableInsertRequestSnafu, UnrecognizedTableOptionSnafu,
};
use crate::expr_factory;
use crate::sql::insert_to_request;
use crate::table::DistTable;
#[derive(Clone)]
@@ -374,7 +374,10 @@ impl DistInstance {
.context(CatalogSnafu)?
.context(TableNotFoundSnafu { table_name: table })?;
let insert_request = insert_to_request(&table, *insert, query_ctx)?;
let insert_request =
SqlHandler::insert_to_request(self.catalog_manager.clone(), *insert, query_ctx)
.await
.context(InvokeDatanodeSnafu)?;
return Ok(Output::AffectedRows(
table.insert(insert_request).await.context(TableSnafu)?,

View File

@@ -590,7 +590,7 @@ CREATE TABLE {table_name} (
.plan(stmt, QueryContext::arc())
.await
.unwrap();
let output = engine.execute(&plan).await.unwrap();
let output = engine.execute(plan, QueryContext::arc()).await.unwrap();
let Output::Stream(stream) = output else { unreachable!() };
let recordbatches = RecordBatches::try_collect(stream).await.unwrap();
let actual = recordbatches.pretty_print().unwrap();

View File

@@ -28,7 +28,6 @@ pub mod postgres;
pub mod prom;
pub mod prometheus;
mod server;
mod sql;
mod table;
#[cfg(test)]
mod tests;

View File

@@ -1,130 +0,0 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use common_error::ext::BoxedError;
use common_error::snafu::ensure;
use datanode::instance::sql::table_idents_to_full_name;
use datatypes::data_type::DataType;
use datatypes::prelude::MutableVector;
use datatypes::schema::ColumnSchema;
use session::context::QueryContextRef;
use snafu::{OptionExt, ResultExt};
use sql::ast::Value as SqlValue;
use sql::statements;
use sql::statements::insert::Insert;
use table::requests::InsertRequest;
use table::TableRef;
use crate::error::{self, ExternalSnafu, Result};
const DEFAULT_PLACEHOLDER_VALUE: &str = "default";
// TODO(fys): Extract the common logic in datanode and frontend in the future.
// This function convert insert statement to an `InsertRequest` to region 0.
pub(crate) fn insert_to_request(
table: &TableRef,
stmt: Insert,
query_ctx: QueryContextRef,
) -> Result<InsertRequest> {
let columns = stmt.columns();
let values = stmt
.values_body()
.context(error::ParseSqlSnafu)?
.context(error::MissingInsertValuesSnafu)?;
let (catalog_name, schema_name, table_name) =
table_idents_to_full_name(stmt.table_name(), query_ctx)
.map_err(BoxedError::new)
.context(ExternalSnafu)?;
let schema = table.schema();
let columns_num = if columns.is_empty() {
schema.column_schemas().len()
} else {
columns.len()
};
let rows_num = values.len();
let mut columns_builders: Vec<(&ColumnSchema, Box<dyn MutableVector>)> =
Vec::with_capacity(columns_num);
if columns.is_empty() {
for column_schema in schema.column_schemas() {
let data_type = &column_schema.data_type;
columns_builders.push((column_schema, data_type.create_mutable_vector(rows_num)));
}
} else {
for column_name in columns {
let column_schema = schema.column_schema_by_name(column_name).with_context(|| {
error::ColumnNotFoundSnafu {
table_name: &table_name,
column_name: column_name.to_string(),
}
})?;
let data_type = &column_schema.data_type;
columns_builders.push((column_schema, data_type.create_mutable_vector(rows_num)));
}
}
for row in values {
ensure!(
row.len() == columns_num,
error::ColumnValuesNumberMismatchSnafu {
columns: columns_num,
values: row.len(),
}
);
for (sql_val, (column_schema, builder)) in row.iter().zip(columns_builders.iter_mut()) {
add_row_to_vector(column_schema, sql_val, builder)?;
}
}
Ok(InsertRequest {
catalog_name,
schema_name,
table_name,
columns_values: columns_builders
.into_iter()
.map(|(cs, mut b)| (cs.name.to_string(), b.to_vector()))
.collect(),
region_number: 0,
})
}
fn add_row_to_vector(
column_schema: &ColumnSchema,
sql_val: &SqlValue,
builder: &mut Box<dyn MutableVector>,
) -> Result<()> {
let value = if replace_default(sql_val) {
column_schema
.create_default()
.context(error::ColumnDefaultValueSnafu {
column: column_schema.name.to_string(),
})?
.context(error::ColumnNoneDefaultValueSnafu {
column: column_schema.name.to_string(),
})?
} else {
statements::sql_value_to_value(&column_schema.name, &column_schema.data_type, sql_val)
.context(error::ParseSqlSnafu)?
};
builder.push_value_ref(value.as_value_ref());
Ok(())
}
fn replace_default(sql_val: &SqlValue) -> bool {
matches!(sql_val, SqlValue::Placeholder(s) if s.to_lowercase() == DEFAULT_PLACEHOLDER_VALUE)
}

View File

@@ -261,7 +261,7 @@ mod tests {
let stat_val = StatValue { stats: vec![stat] }.try_into().unwrap();
let kv = KeyValue {
key: stat_key.clone().into(),
key: stat_key.into(),
value: stat_val,
};

View File

@@ -32,30 +32,27 @@ use std::collections::VecDeque;
///
/// where F is the cumulative distribution function of a normal distribution with mean
/// and standard deviation estimated from historical heartbeat inter-arrival times.
#[cfg_attr(test, derive(Clone))]
pub(crate) struct PhiAccrualFailureDetector {
/// A low threshold is prone to generate many wrong suspicions but ensures a quick detection
/// in the event of a real crash. Conversely, a high threshold generates fewer mistakes but
/// needs more time to detect actual crashes.
threshold: f64,
/// Number of samples to use for calculation of mean and standard deviation of inter-arrival
/// times.
max_sample_size: u32,
threshold: f32,
/// Minimum standard deviation to use for the normal distribution used when calculating phi.
/// Too low standard deviation might result in too much sensitivity for sudden, but normal,
/// deviations in heartbeat inter arrival times.
min_std_deviation_millis: f64,
min_std_deviation_millis: f32,
/// Duration corresponding to number of potentially lost/delayed heartbeats that will be
/// accepted before considering it to be an anomaly.
/// This margin is important to be able to survive sudden, occasional, pauses in heartbeat
/// arrivals, due to for example network drop.
acceptable_heartbeat_pause_millis: i64,
acceptable_heartbeat_pause_millis: u32,
/// Bootstrap the stats with heartbeats that corresponds to this duration, with a rather high
/// standard deviation (since environment is unknown in the beginning).
first_heartbeat_estimate_millis: i64,
first_heartbeat_estimate_millis: u32,
heartbeat_history: HeartbeatHistory,
last_heartbeat_millis: Option<i64>,
@@ -65,14 +62,12 @@ impl Default for PhiAccrualFailureDetector {
fn default() -> Self {
// default configuration is the same as of Akka:
// https://github.com/akka/akka/blob/main/akka-cluster/src/main/resources/reference.conf#L181
let max_sample_size = 1000;
Self {
threshold: 8_f64,
max_sample_size,
min_std_deviation_millis: 100_f64,
threshold: 8_f32,
min_std_deviation_millis: 100_f32,
acceptable_heartbeat_pause_millis: 3000,
first_heartbeat_estimate_millis: 1000,
heartbeat_history: HeartbeatHistory::new(max_sample_size),
heartbeat_history: HeartbeatHistory::new(1000),
last_heartbeat_millis: None,
}
}
@@ -95,28 +90,28 @@ impl PhiAccrualFailureDetector {
// bootstrap with 2 entries with rather high standard deviation
let std_deviation = self.first_heartbeat_estimate_millis / 4;
self.heartbeat_history
.add(self.first_heartbeat_estimate_millis - std_deviation);
.add((self.first_heartbeat_estimate_millis - std_deviation) as _);
self.heartbeat_history
.add(self.first_heartbeat_estimate_millis + std_deviation);
.add((self.first_heartbeat_estimate_millis + std_deviation) as _);
}
let _ = self.last_heartbeat_millis.insert(ts_millis);
}
pub(crate) fn is_available(&self, ts_millis: i64) -> bool {
self.phi(ts_millis) < self.threshold
self.phi(ts_millis) < self.threshold as _
}
/// The suspicion level of the accrual failure detector.
///
/// If a connection does not have any records in failure detector then it is considered healthy.
fn phi(&self, ts_millis: i64) -> f64 {
pub(crate) fn phi(&self, ts_millis: i64) -> f64 {
if let Some(last_heartbeat_millis) = self.last_heartbeat_millis {
let time_diff = ts_millis - last_heartbeat_millis;
let mean = self.heartbeat_history.mean();
let std_deviation = self
.heartbeat_history
.std_deviation()
.max(self.min_std_deviation_millis);
.max(self.min_std_deviation_millis as _);
phi(
time_diff,
@@ -128,6 +123,16 @@ impl PhiAccrualFailureDetector {
0.0
}
}
#[cfg(test)]
pub(crate) fn threshold(&self) -> f32 {
self.threshold
}
#[cfg(test)]
pub(crate) fn acceptable_heartbeat_pause_millis(&self) -> u32 {
self.acceptable_heartbeat_pause_millis
}
}
/// Calculation of phi, derived from the Cumulative distribution function for
@@ -141,6 +146,8 @@ impl PhiAccrualFailureDetector {
/// Usually phi = 1 means likeliness that we will make a mistake is about 10%.
/// The likeliness is about 1% with phi = 2, 0.1% with phi = 3 and so on.
fn phi(time_diff: i64, mean: f64, std_deviation: f64) -> f64 {
assert_ne!(std_deviation, 0.0);
let time_diff = time_diff as f64;
let y = (time_diff - mean) / std_deviation;
let e = (-y * (1.5976 + 0.070566 * y * y)).exp();
@@ -155,8 +162,12 @@ fn phi(time_diff: i64, mean: f64, std_deviation: f64) -> f64 {
/// It is capped by the number of samples specified in `max_sample_size`.
///
/// The stats (mean, variance, std_deviation) are not defined for empty HeartbeatHistory.
#[derive(Clone)]
struct HeartbeatHistory {
/// Number of samples to use for calculation of mean and standard deviation of inter-arrival
/// times.
max_sample_size: u32,
intervals: VecDeque<i64>,
interval_sum: i64,
squared_interval_sum: i64,
@@ -198,7 +209,7 @@ impl HeartbeatHistory {
let oldest = self
.intervals
.pop_front()
.expect("intervals must not empty here");
.expect("intervals must not be empty here");
self.interval_sum -= oldest;
self.squared_interval_sum -= oldest * oldest;
}
@@ -207,42 +218,9 @@ impl HeartbeatHistory {
#[cfg(test)]
mod tests {
use common_time::util::current_time_millis;
use rand::Rng;
use super::*;
#[test]
fn test_heartbeat() {
// Generate 2000 heartbeats start from now. Heartbeat interval is one second, plus some
// random millis.
fn generate_heartbeats() -> Vec<i64> {
let mut rng = rand::thread_rng();
let start = current_time_millis();
(0..2000)
.map(|i| start + i * 1000 + rng.gen_range(0..100))
.collect::<Vec<i64>>()
}
let heartbeats = generate_heartbeats();
let mut fd = PhiAccrualFailureDetector::default();
// feed the failure detector with these heartbeats
heartbeats.iter().for_each(|x| fd.heartbeat(*x));
let start = *heartbeats.last().unwrap();
// Within the "acceptable_heartbeat_pause_millis" period, phi is zero ...
for i in 1..=fd.acceptable_heartbeat_pause_millis / 1000 {
let now = start + i * 1000;
assert_eq!(fd.phi(now), 0.0);
}
// ... then in less than two seconds, phi is above the threshold.
// The same effect can be seen in the diagrams in Akka's document.
let now = start + fd.acceptable_heartbeat_pause_millis + 1000;
assert!(fd.phi(now) < fd.threshold);
let now = start + fd.acceptable_heartbeat_pause_millis + 2000;
assert!(fd.phi(now) > fd.threshold);
}
#[test]
fn test_is_available() {
let ts_millis = current_time_millis();
@@ -254,12 +232,13 @@ mod tests {
fd.heartbeat(ts_millis);
let acceptable_heartbeat_pause_millis = fd.acceptable_heartbeat_pause_millis as i64;
// is available when heartbeat
assert!(fd.is_available(ts_millis));
// is available before heartbeat timeout
assert!(fd.is_available(ts_millis + fd.acceptable_heartbeat_pause_millis / 2));
assert!(fd.is_available(ts_millis + acceptable_heartbeat_pause_millis / 2));
// is not available after heartbeat timeout
assert!(!fd.is_available(ts_millis + fd.acceptable_heartbeat_pause_millis * 2));
assert!(!fd.is_available(ts_millis + acceptable_heartbeat_pause_millis * 2));
}
#[test]
@@ -286,14 +265,15 @@ mod tests {
fd.heartbeat(ts_millis);
let acceptable_heartbeat_pause_millis = fd.acceptable_heartbeat_pause_millis as i64;
// phi == 0 when heartbeat
assert_eq!(fd.phi(ts_millis), 0.0);
// phi < threshold before heartbeat timeout
let now = ts_millis + fd.acceptable_heartbeat_pause_millis / 2;
assert!(fd.phi(now) < fd.threshold);
let now = ts_millis + acceptable_heartbeat_pause_millis / 2;
assert!(fd.phi(now) < fd.threshold as _);
// phi >= threshold after heartbeat timeout
let now = ts_millis + fd.acceptable_heartbeat_pause_millis * 2;
assert!(fd.phi(now) >= fd.threshold);
let now = ts_millis + acceptable_heartbeat_pause_millis * 2;
assert!(fd.phi(now) >= fd.threshold as _);
}
// The following test cases are port from Akka's test:
@@ -349,7 +329,6 @@ mod tests {
fn test_return_phi_of_0_on_startup_when_no_heartbeats() {
let fd = PhiAccrualFailureDetector {
threshold: 8.0,
max_sample_size: 1000,
min_std_deviation_millis: 100.0,
acceptable_heartbeat_pause_millis: 0,
first_heartbeat_estimate_millis: 1000,
@@ -364,7 +343,6 @@ mod tests {
fn test_return_phi_based_on_guess_when_only_one_heartbeat() {
let mut fd = PhiAccrualFailureDetector {
threshold: 8.0,
max_sample_size: 1000,
min_std_deviation_millis: 100.0,
acceptable_heartbeat_pause_millis: 0,
first_heartbeat_estimate_millis: 1000,
@@ -381,7 +359,6 @@ mod tests {
fn test_return_phi_using_first_interval_after_second_heartbeat() {
let mut fd = PhiAccrualFailureDetector {
threshold: 8.0,
max_sample_size: 1000,
min_std_deviation_millis: 100.0,
acceptable_heartbeat_pause_millis: 0,
first_heartbeat_estimate_millis: 1000,
@@ -398,7 +375,6 @@ mod tests {
fn test_is_available_after_a_series_of_successful_heartbeats() {
let mut fd = PhiAccrualFailureDetector {
threshold: 8.0,
max_sample_size: 1000,
min_std_deviation_millis: 100.0,
acceptable_heartbeat_pause_millis: 0,
first_heartbeat_estimate_millis: 1000,
@@ -417,7 +393,6 @@ mod tests {
fn test_is_not_available_if_heartbeat_are_missed() {
let mut fd = PhiAccrualFailureDetector {
threshold: 3.0,
max_sample_size: 1000,
min_std_deviation_millis: 100.0,
acceptable_heartbeat_pause_millis: 0,
first_heartbeat_estimate_millis: 1000,
@@ -436,7 +411,6 @@ mod tests {
) {
let mut fd = PhiAccrualFailureDetector {
threshold: 8.0,
max_sample_size: 1000,
min_std_deviation_millis: 100.0,
acceptable_heartbeat_pause_millis: 3000,
first_heartbeat_estimate_millis: 1000,
@@ -476,7 +450,6 @@ mod tests {
fn test_accept_some_configured_missing_heartbeats() {
let mut fd = PhiAccrualFailureDetector {
threshold: 8.0,
max_sample_size: 1000,
min_std_deviation_millis: 100.0,
acceptable_heartbeat_pause_millis: 3000,
first_heartbeat_estimate_millis: 1000,
@@ -496,7 +469,6 @@ mod tests {
fn test_fail_after_configured_acceptable_missing_heartbeats() {
let mut fd = PhiAccrualFailureDetector {
threshold: 8.0,
max_sample_size: 1000,
min_std_deviation_millis: 100.0,
acceptable_heartbeat_pause_millis: 3000,
first_heartbeat_estimate_millis: 1000,
@@ -518,7 +490,6 @@ mod tests {
fn test_use_max_sample_size_heartbeats() {
let mut fd = PhiAccrualFailureDetector {
threshold: 8.0,
max_sample_size: 3,
min_std_deviation_millis: 100.0,
acceptable_heartbeat_pause_millis: 0,
first_heartbeat_estimate_millis: 1000,

View File

@@ -14,6 +14,7 @@
pub use check_leader_handler::CheckLeaderHandler;
pub use collect_stats_handler::CollectStatsHandler;
pub use failure_handler::RegionFailureHandler;
pub use keep_lease_handler::KeepLeaseHandler;
pub use on_leader_start::OnLeaderStartHandler;
pub use persist_stats_handler::PersistStatsHandler;
@@ -21,6 +22,7 @@ pub use response_header_handler::ResponseHeaderHandler;
mod check_leader_handler;
mod collect_stats_handler;
mod failure_handler;
mod instruction;
mod keep_lease_handler;
pub mod node_stat;
@@ -54,8 +56,8 @@ pub trait HeartbeatHandler: Send + Sync {
#[derive(Debug, Default)]
pub struct HeartbeatAccumulator {
pub header: Option<ResponseHeader>,
pub stats: Vec<Stat>,
pub instructions: Vec<Instruction>,
pub stat: Option<Stat>,
}
impl HeartbeatAccumulator {

View File

@@ -12,39 +12,15 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::collections::VecDeque;
use api::v1::meta::HeartbeatRequest;
use common_telemetry::debug;
use dashmap::mapref::entry::Entry;
use dashmap::DashMap;
use super::node_stat::Stat;
use crate::error::Result;
use crate::handler::{HeartbeatAccumulator, HeartbeatHandler};
use crate::metasrv::Context;
type StatKey = (u64, u64);
pub struct CollectStatsHandler {
max_cached_stats_per_key: usize,
cache: DashMap<StatKey, VecDeque<Stat>>,
}
impl Default for CollectStatsHandler {
fn default() -> Self {
Self::new(10)
}
}
impl CollectStatsHandler {
pub fn new(max_cached_stats_per_key: usize) -> Self {
Self {
max_cached_stats_per_key,
cache: DashMap::new(),
}
}
}
pub struct CollectStatsHandler;
#[async_trait::async_trait]
impl HeartbeatHandler for CollectStatsHandler {
@@ -60,21 +36,7 @@ impl HeartbeatHandler for CollectStatsHandler {
match Stat::try_from(req.clone()) {
Ok(stat) => {
let key = (stat.cluster_id, stat.id);
match self.cache.entry(key) {
Entry::Occupied(mut e) => {
let deque = e.get_mut();
deque.push_front(stat);
if deque.len() >= self.max_cached_stats_per_key {
acc.stats = deque.drain(..).collect();
}
}
Entry::Vacant(e) => {
let mut stat_vec = VecDeque::with_capacity(self.max_cached_stats_per_key);
stat_vec.push_front(stat);
e.insert(stat_vec);
}
}
let _ = acc.stat.insert(stat);
}
Err(_) => {
debug!("Incomplete heartbeat data: {:?}", req);

View File

@@ -0,0 +1,151 @@
// 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 runner;
use api::v1::meta::HeartbeatRequest;
use async_trait::async_trait;
use crate::error::Result;
use crate::handler::failure_handler::runner::{FailureDetectControl, FailureDetectRunner};
use crate::handler::{HeartbeatAccumulator, HeartbeatHandler};
use crate::metasrv::{Context, ElectionRef};
#[derive(Eq, Hash, PartialEq, Clone)]
pub(crate) struct RegionIdent {
catalog: String,
schema: String,
table: String,
region_id: u64,
}
// TODO(LFC): TBC
pub(crate) struct DatanodeHeartbeat {
#[allow(dead_code)]
cluster_id: u64,
#[allow(dead_code)]
node_id: u64,
region_idents: Vec<RegionIdent>,
heartbeat_time: i64,
}
pub struct RegionFailureHandler {
failure_detect_runner: FailureDetectRunner,
}
impl RegionFailureHandler {
pub fn new(election: Option<ElectionRef>) -> Self {
Self {
failure_detect_runner: FailureDetectRunner::new(election),
}
}
pub async fn start(&mut self) {
self.failure_detect_runner.start().await;
}
}
#[async_trait]
impl HeartbeatHandler for RegionFailureHandler {
async fn handle(
&self,
_: &HeartbeatRequest,
ctx: &mut Context,
acc: &mut HeartbeatAccumulator,
) -> Result<()> {
if ctx.is_infancy {
self.failure_detect_runner
.send_control(FailureDetectControl::Purge)
.await;
}
if ctx.is_skip_all() {
return Ok(());
}
let Some(stat) = acc.stat.as_ref() else { return Ok(()) };
let heartbeat = DatanodeHeartbeat {
cluster_id: stat.cluster_id,
node_id: stat.id,
region_idents: stat
.region_stats
.iter()
.map(|x| RegionIdent {
catalog: x.catalog.clone(),
schema: x.schema.clone(),
table: x.table.clone(),
region_id: x.id,
})
.collect(),
heartbeat_time: stat.timestamp_millis,
};
self.failure_detect_runner.send_heartbeat(heartbeat).await;
Ok(())
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::handler::node_stat::{RegionStat, Stat};
use crate::metasrv::builder::MetaSrvBuilder;
#[tokio::test(flavor = "multi_thread")]
async fn test_handle_heartbeat() {
let mut handler = RegionFailureHandler::new(None);
handler.start().await;
let req = &HeartbeatRequest::default();
let builder = MetaSrvBuilder::new();
let metasrv = builder.build().await;
let mut ctx = metasrv.new_ctx();
ctx.is_infancy = false;
let acc = &mut HeartbeatAccumulator::default();
fn new_region_stat(region_id: u64) -> RegionStat {
RegionStat {
id: region_id,
catalog: "a".to_string(),
schema: "b".to_string(),
table: "c".to_string(),
rcus: 0,
wcus: 0,
approximate_bytes: 0,
approximate_rows: 0,
}
}
acc.stat = Some(Stat {
cluster_id: 1,
id: 42,
region_stats: vec![new_region_stat(1), new_region_stat(2), new_region_stat(3)],
timestamp_millis: 1000,
..Default::default()
});
handler.handle(req, &mut ctx, acc).await.unwrap();
let dump = handler.failure_detect_runner.dump().await;
assert_eq!(dump.iter().collect::<Vec<_>>().len(), 3);
// infancy makes heartbeats re-accumulated
ctx.is_infancy = true;
acc.stat = None;
handler.handle(req, &mut ctx, acc).await.unwrap();
let dump = handler.failure_detect_runner.dump().await;
assert_eq!(dump.iter().collect::<Vec<_>>().len(), 0);
}
}

View File

@@ -0,0 +1,309 @@
// 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::ops::DerefMut;
use std::sync::Arc;
use std::time::{Duration, Instant};
use common_telemetry::error;
use common_time::util::current_time_millis;
use dashmap::mapref::multiple::RefMulti;
use dashmap::DashMap;
use tokio::sync::mpsc;
use tokio::sync::mpsc::{Receiver, Sender};
use tokio::task::JoinHandle;
use crate::failure_detector::PhiAccrualFailureDetector;
use crate::handler::failure_handler::{DatanodeHeartbeat, RegionIdent};
use crate::metasrv::ElectionRef;
pub(crate) enum FailureDetectControl {
Purge,
#[cfg(test)]
Dump(tokio::sync::oneshot::Sender<FailureDetectorContainer>),
}
pub(crate) struct FailureDetectRunner {
election: Option<ElectionRef>,
heartbeat_tx: Sender<DatanodeHeartbeat>,
heartbeat_rx: Option<Receiver<DatanodeHeartbeat>>,
control_tx: Sender<FailureDetectControl>,
control_rx: Option<Receiver<FailureDetectControl>>,
receiver_handle: Option<JoinHandle<()>>,
runner_handle: Option<JoinHandle<()>>,
}
impl FailureDetectRunner {
pub(crate) fn new(election: Option<ElectionRef>) -> Self {
let (heartbeat_tx, heartbeat_rx) = mpsc::channel::<DatanodeHeartbeat>(1024);
let (control_tx, control_rx) = mpsc::channel::<FailureDetectControl>(1024);
Self {
election,
heartbeat_tx,
heartbeat_rx: Some(heartbeat_rx),
control_tx,
control_rx: Some(control_rx),
receiver_handle: None,
runner_handle: None,
}
}
pub(crate) async fn send_heartbeat(&self, heartbeat: DatanodeHeartbeat) {
if let Err(e) = self.heartbeat_tx.send(heartbeat).await {
error!("FailureDetectRunner is stop receiving heartbeats: {}", e)
}
}
pub(crate) async fn send_control(&self, control: FailureDetectControl) {
if let Err(e) = self.control_tx.send(control).await {
error!("FailureDetectRunner is stop receiving controls: {}", e)
}
}
pub(crate) async fn start(&mut self) {
let failure_detectors = Arc::new(FailureDetectorContainer(DashMap::new()));
self.start_with(failure_detectors).await
}
async fn start_with(&mut self, failure_detectors: Arc<FailureDetectorContainer>) {
let Some(mut heartbeat_rx) = self.heartbeat_rx.take() else { return };
let Some(mut control_rx) = self.control_rx.take() else { return };
let container = failure_detectors.clone();
let receiver_handle = common_runtime::spawn_bg(async move {
loop {
tokio::select! {
Some(control) = control_rx.recv() => {
match control {
FailureDetectControl::Purge => container.clear(),
#[cfg(test)]
FailureDetectControl::Dump(tx) => {
// Drain any heartbeats that are not handled before dump.
while let Ok(heartbeat) = heartbeat_rx.try_recv() {
for ident in heartbeat.region_idents {
let mut detector = container.get_failure_detector(ident);
detector.heartbeat(heartbeat.heartbeat_time);
}
}
let _ = tx.send(container.dump());
}
}
}
Some(heartbeat) = heartbeat_rx.recv() => {
for ident in heartbeat.region_idents {
let mut detector = container.get_failure_detector(ident);
detector.heartbeat(heartbeat.heartbeat_time);
}
}
}
}
});
self.receiver_handle = Some(receiver_handle);
let election = self.election.clone();
let runner_handle = common_runtime::spawn_bg(async move {
loop {
let start = Instant::now();
let is_leader = election.as_ref().map(|x| x.is_leader()).unwrap_or(true);
if is_leader {
for e in failure_detectors.iter() {
if e.failure_detector().is_available(current_time_millis()) {
// TODO(LFC): TBC
}
}
}
let elapsed = Instant::now().duration_since(start);
if let Some(sleep) = Duration::from_secs(1).checked_sub(elapsed) {
tokio::time::sleep(sleep).await;
} // else the elapsed time is exceeding one second, we should continue working immediately
}
});
self.runner_handle = Some(runner_handle);
}
#[cfg(test)]
fn abort(&mut self) {
let Some(handle) = self.receiver_handle.take() else { return };
handle.abort();
let Some(handle) = self.runner_handle.take() else { return };
handle.abort();
}
#[cfg(test)]
pub(crate) async fn dump(&self) -> FailureDetectorContainer {
let (tx, rx) = tokio::sync::oneshot::channel();
self.send_control(FailureDetectControl::Dump(tx)).await;
rx.await.unwrap()
}
}
pub(crate) struct FailureDetectorEntry<'a> {
e: RefMulti<'a, RegionIdent, PhiAccrualFailureDetector>,
}
impl FailureDetectorEntry<'_> {
fn failure_detector(&self) -> &PhiAccrualFailureDetector {
self.e.value()
}
}
pub(crate) struct FailureDetectorContainer(DashMap<RegionIdent, PhiAccrualFailureDetector>);
impl FailureDetectorContainer {
fn get_failure_detector(
&self,
ident: RegionIdent,
) -> impl DerefMut<Target = PhiAccrualFailureDetector> + '_ {
self.0
.entry(ident)
.or_insert_with(PhiAccrualFailureDetector::default)
}
pub(crate) fn iter(&self) -> Box<dyn Iterator<Item = FailureDetectorEntry> + '_> {
Box::new(self.0.iter().map(move |e| FailureDetectorEntry { e })) as _
}
fn clear(&self) {
self.0.clear()
}
#[cfg(test)]
fn dump(&self) -> FailureDetectorContainer {
let mut m = DashMap::with_capacity(self.0.len());
m.extend(self.0.iter().map(|x| (x.key().clone(), x.value().clone())));
Self(m)
}
}
#[cfg(test)]
mod tests {
use rand::Rng;
use super::*;
#[test]
fn test_default_failure_detector_container() {
let container = FailureDetectorContainer(DashMap::new());
let ident = RegionIdent {
catalog: "a".to_string(),
schema: "b".to_string(),
table: "c".to_string(),
region_id: 1,
};
let _ = container.get_failure_detector(ident.clone());
assert!(container.0.contains_key(&ident));
{
let mut iter = container.iter();
assert!(iter.next().is_some());
assert!(iter.next().is_none());
}
container.clear();
assert!(container.0.is_empty());
}
#[tokio::test(flavor = "multi_thread")]
async fn test_control() {
let container = FailureDetectorContainer(DashMap::new());
let ident = RegionIdent {
catalog: "a".to_string(),
schema: "b".to_string(),
table: "c".to_string(),
region_id: 1,
};
container.get_failure_detector(ident.clone());
let mut runner = FailureDetectRunner::new(None);
runner.start_with(Arc::new(container)).await;
let dump = runner.dump().await;
assert_eq!(dump.iter().collect::<Vec<_>>().len(), 1);
runner.send_control(FailureDetectControl::Purge).await;
let dump = runner.dump().await;
assert_eq!(dump.iter().collect::<Vec<_>>().len(), 0);
runner.abort();
}
#[tokio::test(flavor = "multi_thread")]
async fn test_heartbeat() {
let mut runner = FailureDetectRunner::new(None);
runner.start().await;
// Generate 2000 heartbeats start from now. Heartbeat interval is one second, plus some random millis.
fn generate_heartbeats(node_id: u64, region_ids: Vec<u64>) -> Vec<DatanodeHeartbeat> {
let mut rng = rand::thread_rng();
let start = current_time_millis();
(0..2000)
.map(|i| DatanodeHeartbeat {
cluster_id: 1,
node_id,
region_idents: region_ids
.iter()
.map(|&region_id| RegionIdent {
catalog: "a".to_string(),
schema: "b".to_string(),
table: "c".to_string(),
region_id,
})
.collect(),
heartbeat_time: start + i * 1000 + rng.gen_range(0..100),
})
.collect::<Vec<_>>()
}
let heartbeats = generate_heartbeats(100, vec![1, 2, 3]);
let last_heartbeat_time = heartbeats.last().unwrap().heartbeat_time;
for heartbeat in heartbeats {
runner.send_heartbeat(heartbeat).await;
}
let dump = runner.dump().await;
let failure_detectors = dump.iter().collect::<Vec<_>>();
assert_eq!(failure_detectors.len(), 3);
failure_detectors.iter().for_each(|e| {
let fd = e.failure_detector();
let acceptable_heartbeat_pause_millis = fd.acceptable_heartbeat_pause_millis() as i64;
let start = last_heartbeat_time;
// Within the "acceptable_heartbeat_pause_millis" period, phi is zero ...
for i in 1..=acceptable_heartbeat_pause_millis / 1000 {
let now = start + i * 1000;
assert_eq!(fd.phi(now), 0.0);
}
// ... then in less than two seconds, phi is above the threshold.
// The same effect can be seen in the diagrams in Akka's document.
let now = start + acceptable_heartbeat_pause_millis + 1000;
assert!(fd.phi(now) < fd.threshold() as _);
let now = start + acceptable_heartbeat_pause_millis + 2000;
assert!(fd.phi(now) > fd.threshold() as _);
});
runner.abort();
}
}

View File

@@ -31,6 +31,7 @@ impl HeartbeatHandler for OnLeaderStartHandler {
) -> Result<()> {
if let Some(election) = &ctx.election {
if election.in_infancy() {
ctx.is_infancy = true;
ctx.reset_in_memory();
}
}

View File

@@ -13,14 +13,20 @@
// limitations under the License.
use api::v1::meta::{HeartbeatRequest, PutRequest};
use dashmap::DashMap;
use crate::error::Result;
use crate::handler::node_stat::Stat;
use crate::handler::{HeartbeatAccumulator, HeartbeatHandler};
use crate::keys::StatValue;
use crate::keys::{StatKey, StatValue};
use crate::metasrv::Context;
const MAX_CACHED_STATS_PER_KEY: usize = 10;
#[derive(Default)]
pub struct PersistStatsHandler;
pub struct PersistStatsHandler {
stats_cache: DashMap<StatKey, Vec<Stat>>,
}
#[async_trait::async_trait]
impl HeartbeatHandler for PersistStatsHandler {
@@ -30,18 +36,25 @@ impl HeartbeatHandler for PersistStatsHandler {
ctx: &mut Context,
acc: &mut HeartbeatAccumulator,
) -> Result<()> {
if ctx.is_skip_all() || acc.stats.is_empty() {
if ctx.is_skip_all() {
return Ok(());
}
let stats = &mut acc.stats;
let key = match stats.get(0) {
Some(stat) => stat.stat_key(),
None => return Ok(()),
};
let Some(stat) = acc.stat.take() else { return Ok(()) };
// take stats from &mut acc.stats, avoid clone of vec
let stats = std::mem::take(stats);
let key = stat.stat_key();
let mut entry = self
.stats_cache
.entry(key)
.or_insert_with(|| Vec::with_capacity(MAX_CACHED_STATS_PER_KEY));
let stats = entry.value_mut();
stats.push(stat);
if stats.len() < MAX_CACHED_STATS_PER_KEY {
return Ok(());
}
let stats = stats.drain(..).collect();
let val = StatValue { stats };
@@ -65,7 +78,6 @@ mod tests {
use api::v1::meta::RangeRequest;
use super::*;
use crate::handler::node_stat::Stat;
use crate::keys::StatKey;
use crate::service::store::memory::MemStore;
@@ -83,24 +95,23 @@ mod tests {
catalog: None,
schema: None,
table: None,
is_infancy: false,
};
let req = HeartbeatRequest::default();
let mut acc = HeartbeatAccumulator {
stats: vec![Stat {
cluster_id: 3,
id: 101,
region_num: Some(100),
let handler = PersistStatsHandler::default();
for i in 1..=MAX_CACHED_STATS_PER_KEY {
let mut acc = HeartbeatAccumulator {
stat: Some(Stat {
cluster_id: 3,
id: 101,
region_num: Some(i as _),
..Default::default()
}),
..Default::default()
}],
..Default::default()
};
let stats_handler = PersistStatsHandler;
stats_handler
.handle(&req, &mut ctx, &mut acc)
.await
.unwrap();
};
handler.handle(&req, &mut ctx, &mut acc).await.unwrap();
}
let key = StatKey {
cluster_id: 3,
@@ -124,7 +135,7 @@ mod tests {
let val: StatValue = kv.value.clone().try_into().unwrap();
assert_eq!(1, val.stats.len());
assert_eq!(Some(100), val.stats[0].region_num);
assert_eq!(10, val.stats.len());
assert_eq!(Some(1), val.stats[0].region_num);
}
}

View File

@@ -65,6 +65,7 @@ mod tests {
catalog: None,
schema: None,
table: None,
is_infancy: false,
};
let req = HeartbeatRequest {

View File

@@ -178,7 +178,7 @@ pub(crate) fn to_removed_key(key: &str) -> String {
format!("{REMOVED_PREFIX}-{key}")
}
#[derive(Eq, PartialEq, Debug, Clone, Hash)]
#[derive(Eq, PartialEq, Debug, Clone, Hash, Copy)]
pub struct StatKey {
pub cluster_id: u64,
pub node_id: u64,

View File

@@ -17,8 +17,6 @@ pub mod bootstrap;
pub mod cluster;
pub mod election;
pub mod error;
// TODO(LFC): TBC
#[allow(dead_code)]
mod failure_detector;
pub mod handler;
pub mod keys;

View File

@@ -66,6 +66,7 @@ pub struct Context {
pub catalog: Option<String>,
pub schema: Option<String>,
pub table: Option<String>,
pub is_infancy: bool,
}
impl Context {
@@ -199,6 +200,7 @@ impl MetaSrv {
catalog: None,
schema: None,
table: None,
is_infancy: false,
}
}
}

View File

@@ -18,7 +18,7 @@ use std::sync::Arc;
use crate::cluster::MetaPeerClient;
use crate::handler::{
CheckLeaderHandler, CollectStatsHandler, HeartbeatHandlerGroup, KeepLeaseHandler,
OnLeaderStartHandler, PersistStatsHandler, ResponseHeaderHandler,
OnLeaderStartHandler, PersistStatsHandler, RegionFailureHandler, ResponseHeaderHandler,
};
use crate::lock::DistLockRef;
use crate::metasrv::{ElectionRef, MetaSrv, MetaSrvOptions, SelectorRef, TABLE_ID_SEQ};
@@ -118,6 +118,9 @@ impl MetaSrvBuilder {
let handler_group = match handler_group {
Some(handler_group) => handler_group,
None => {
let mut region_failure_handler = RegionFailureHandler::new(election.clone());
region_failure_handler.start().await;
let group = HeartbeatHandlerGroup::default();
let keep_lease_handler = KeepLeaseHandler::new(kv_store.clone());
group.add_handler(ResponseHeaderHandler::default()).await;
@@ -127,7 +130,8 @@ impl MetaSrvBuilder {
group.add_handler(keep_lease_handler).await;
group.add_handler(CheckLeaderHandler::default()).await;
group.add_handler(OnLeaderStartHandler::default()).await;
group.add_handler(CollectStatsHandler::default()).await;
group.add_handler(CollectStatsHandler).await;
group.add_handler(region_failure_handler).await;
group.add_handler(PersistStatsHandler::default()).await;
group
}

View File

@@ -13,8 +13,11 @@
// limitations under the License.
mod aggr_over_time;
mod changes;
mod deriv;
mod idelta;
mod increase;
mod resets;
#[cfg(test)]
mod test_util;

View File

@@ -0,0 +1,16 @@
// 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.
//! Implementation of [`reset`](https://prometheus.io/docs/prometheus/latest/querying/functions/#changes) in PromQL. Refer to the [original
//! implementation](https://github.com/prometheus/prometheus/blob/main/promql/functions.go#L1023-L1040).

View File

@@ -0,0 +1,16 @@
// 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.
//! Implementation of [`reset`](https://prometheus.io/docs/prometheus/latest/querying/functions/#deriv) in PromQL. Refer to the [original
//! implementation](https://github.com/prometheus/prometheus/blob/90b2f7a540b8a70d8d81372e6692dcbb67ccbaaa/promql/functions.go#L839-L856).

View File

@@ -0,0 +1,16 @@
// 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.
//! Implementation of [`reset`](https://prometheus.io/docs/prometheus/latest/querying/functions/#resets) in PromQL. Refer to the [original
//! implementation](https://github.com/prometheus/prometheus/blob/90b2f7a540b8a70d8d81372e6692dcbb67ccbaaa/promql/functions.go#L1004-L1021).

View File

@@ -1597,7 +1597,6 @@ mod test {
}
#[tokio::test]
#[ignore = "wait for https://github.com/apache/arrow-datafusion/issues/5513"]
async fn increase_aggr() {
let query = "increase(some_metric[5m])";
let expected = String::from(
@@ -1631,7 +1630,6 @@ mod test {
}
#[tokio::test]
#[ignore = "wait for https://github.com/apache/arrow-datafusion/issues/5513"]
async fn count_over_time() {
let query = "count_over_time(some_metric[5m])";
let expected = String::from(

View File

@@ -18,6 +18,7 @@ mod catalog_adapter;
mod error;
mod planner;
use std::collections::HashMap;
use std::sync::Arc;
use async_trait::async_trait;
@@ -33,12 +34,23 @@ use common_recordbatch::{EmptyRecordBatchStream, SendableRecordBatchStream};
use common_telemetry::timer;
use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
use datafusion::physical_plan::ExecutionPlan;
use datafusion_common::ResolvedTableReference;
use datafusion_expr::{DmlStatement, LogicalPlan as DfLogicalPlan, WriteOp};
use datatypes::prelude::VectorRef;
use datatypes::schema::Schema;
use snafu::{OptionExt, ResultExt};
use futures_util::StreamExt;
use session::context::QueryContextRef;
use snafu::{ensure, OptionExt, ResultExt};
use table::requests::{DeleteRequest, InsertRequest};
use table::TableRef;
pub use crate::datafusion::catalog_adapter::DfCatalogListAdapter;
pub use crate::datafusion::planner::DfContextProviderAdapter;
use crate::error::{DataFusionSnafu, QueryExecutionSnafu, Result};
use crate::error::{
CatalogNotFoundSnafu, CatalogSnafu, CreateRecordBatchSnafu, DataFusionSnafu,
MissingTimestampColumnSnafu, QueryExecutionSnafu, Result, SchemaNotFoundSnafu,
TableNotFoundSnafu, UnsupportedExprSnafu,
};
use crate::executor::QueryExecutor;
use crate::logical_optimizer::LogicalOptimizer;
use crate::physical_optimizer::PhysicalOptimizer;
@@ -56,6 +68,145 @@ impl DatafusionQueryEngine {
pub fn new(state: Arc<QueryEngineState>) -> Self {
Self { state }
}
async fn exec_query_plan(&self, plan: LogicalPlan) -> Result<Output> {
let mut ctx = QueryEngineContext::new(self.state.session_state());
// `create_physical_plan` will optimize logical plan internally
let physical_plan = self.create_physical_plan(&mut ctx, &plan).await?;
let physical_plan = self.optimize_physical_plan(&mut ctx, physical_plan)?;
Ok(Output::Stream(self.execute_stream(&ctx, &physical_plan)?))
}
async fn exec_dml_statement(
&self,
dml: DmlStatement,
query_ctx: QueryContextRef,
) -> Result<Output> {
ensure!(
matches!(dml.op, WriteOp::Insert | WriteOp::Delete),
UnsupportedExprSnafu {
name: format!("DML op {}", dml.op),
}
);
let default_catalog = query_ctx.current_catalog();
let default_schema = query_ctx.current_schema();
let table_name = dml
.table_name
.as_table_reference()
.resolve(&default_catalog, &default_schema);
let table = self.find_table(&table_name).await?;
let output = self
.exec_query_plan(LogicalPlan::DfPlan((*dml.input).clone()))
.await?;
let mut stream = match output {
Output::RecordBatches(batches) => batches.as_stream(),
Output::Stream(stream) => stream,
_ => unreachable!(),
};
let mut affected_rows = 0;
while let Some(batch) = stream.next().await {
let batch = batch.context(CreateRecordBatchSnafu)?;
let column_vectors = batch
.column_vectors(&table_name.to_string(), table.schema())
.map_err(BoxedError::new)
.context(QueryExecutionSnafu)?;
let rows = match dml.op {
WriteOp::Insert => Self::insert(&table_name, &table, column_vectors).await?,
WriteOp::Delete => Self::delete(&table_name, &table, column_vectors).await?,
_ => unreachable!("guarded by the 'ensure!' at the beginning"),
};
affected_rows += rows;
}
Ok(Output::AffectedRows(affected_rows))
}
async fn delete<'a>(
table_name: &ResolvedTableReference<'a>,
table: &TableRef,
column_vectors: HashMap<String, VectorRef>,
) -> Result<usize> {
let table_schema = table.schema();
let ts_column = table_schema
.timestamp_column()
.map(|x| &x.name)
.with_context(|| MissingTimestampColumnSnafu {
table_name: table_name.to_string(),
})?;
let table_info = table.table_info();
let rowkey_columns = table_info
.meta
.row_key_column_names()
.collect::<Vec<&String>>();
let column_vectors = column_vectors
.into_iter()
.filter(|x| &x.0 == ts_column || rowkey_columns.contains(&&x.0))
.collect::<HashMap<_, _>>();
let request = DeleteRequest {
key_column_values: column_vectors,
};
table
.delete(request)
.await
.map_err(BoxedError::new)
.context(QueryExecutionSnafu)
}
async fn insert<'a>(
table_name: &ResolvedTableReference<'a>,
table: &TableRef,
column_vectors: HashMap<String, VectorRef>,
) -> Result<usize> {
let request = InsertRequest {
catalog_name: table_name.catalog.to_string(),
schema_name: table_name.schema.to_string(),
table_name: table_name.table.to_string(),
columns_values: column_vectors,
region_number: 0,
};
table
.insert(request)
.await
.map_err(BoxedError::new)
.context(QueryExecutionSnafu)
}
async fn find_table(&self, table_name: &ResolvedTableReference<'_>) -> Result<TableRef> {
let catalog_name = table_name.catalog.as_ref();
let schema_name = table_name.schema.as_ref();
let table_name = table_name.table.as_ref();
let catalog = self
.state
.catalog_list()
.catalog(catalog_name)
.context(CatalogSnafu)?
.context(CatalogNotFoundSnafu {
catalog: catalog_name,
})?;
let schema =
catalog
.schema(schema_name)
.context(CatalogSnafu)?
.context(SchemaNotFoundSnafu {
schema: schema_name,
})?;
let table = schema
.table(table_name)
.await
.context(CatalogSnafu)?
.context(TableNotFoundSnafu { table: table_name })?;
Ok(table)
}
}
#[async_trait]
@@ -75,14 +226,13 @@ impl QueryEngine for DatafusionQueryEngine {
optimised_plan.schema()
}
async fn execute(&self, plan: &LogicalPlan) -> Result<Output> {
let logical_plan = self.optimize(plan)?;
let mut ctx = QueryEngineContext::new(self.state.session_state());
let physical_plan = self.create_physical_plan(&mut ctx, &logical_plan).await?;
let physical_plan = self.optimize_physical_plan(&mut ctx, physical_plan)?;
Ok(Output::Stream(self.execute_stream(&ctx, &physical_plan)?))
async fn execute(&self, plan: LogicalPlan, query_ctx: QueryContextRef) -> Result<Output> {
match plan {
LogicalPlan::DfPlan(DfLogicalPlan::Dml(dml)) => {
self.exec_dml_statement(dml, query_ctx).await
}
_ => self.exec_query_plan(plan).await,
}
}
fn register_udf(&self, udf: ScalarUdf) {
@@ -292,11 +442,11 @@ mod tests {
let stmt = QueryLanguageParser::parse_sql(sql).unwrap();
let plan = engine
.planner()
.plan(stmt, Arc::new(QueryContext::new()))
.plan(stmt, QueryContext::arc())
.await
.unwrap();
let output = engine.execute(&plan).await.unwrap();
let output = engine.execute(plan, QueryContext::arc()).await.unwrap();
match output {
Output::Stream(recordbatch) => {

View File

@@ -72,8 +72,11 @@ pub enum Error {
#[snafu(display("The SQL string has multiple statements, query: {}", query))]
MultipleStatements { query: String, backtrace: Backtrace },
#[snafu(display("Failed to convert datatype: {}", source))]
Datatype { source: datatypes::error::Error },
#[snafu(display("Failed to convert Datafusion schema: {}", source))]
ConvertDatafusionSchema {
#[snafu(backtrace)]
source: datatypes::error::Error,
},
#[snafu(display("Failed to parse timestamp `{}`: {}", raw, source))]
ParseTimestamp {
@@ -107,6 +110,12 @@ pub enum Error {
source: DataFusionError,
backtrace: Backtrace,
},
#[snafu(display("Timestamp column for table '{table_name}' is missing!"))]
MissingTimestampColumn {
table_name: String,
backtrace: Backtrace,
},
}
impl ErrorExt for Error {
@@ -123,11 +132,12 @@ impl ErrorExt for Error {
| ParseFloat { .. } => StatusCode::InvalidArguments,
QueryAccessDenied { .. } => StatusCode::AccessDenied,
Catalog { source } => source.status_code(),
VectorComputation { source } => source.status_code(),
VectorComputation { source } | ConvertDatafusionSchema { source } => {
source.status_code()
}
CreateRecordBatch { source } => source.status_code(),
Datatype { source } => source.status_code(),
QueryExecution { source } | QueryPlan { source } => source.status_code(),
DataFusion { .. } => StatusCode::Internal,
DataFusion { .. } | MissingTimestampColumn { .. } => StatusCode::Internal,
Sql { source } => source.status_code(),
PlanSql { .. } => StatusCode::PlanQuery,
}

View File

@@ -18,7 +18,7 @@ use datafusion_expr::LogicalPlan as DfLogicalPlan;
use datatypes::schema::Schema;
use snafu::ResultExt;
use crate::error::Result;
use crate::error::{ConvertDatafusionSchemaSnafu, Result};
/// A LogicalPlan represents the different types of relational
/// operators (such as Projection, Filter, etc) and can be created by
@@ -42,7 +42,7 @@ impl LogicalPlan {
df_schema
.clone()
.try_into()
.context(crate::error::DatatypeSnafu)
.context(ConvertDatafusionSchemaSnafu)
}
}
}

View File

@@ -56,7 +56,7 @@ pub trait QueryEngine: Send + Sync {
async fn describe(&self, plan: LogicalPlan) -> Result<Schema>;
async fn execute(&self, plan: &LogicalPlan) -> Result<Output>;
async fn execute(&self, plan: LogicalPlan, query_ctx: QueryContextRef) -> Result<Output>;
fn register_udf(&self, udf: ScalarUdf);

View File

@@ -41,7 +41,7 @@ async fn exec_selection(engine: QueryEngineRef, sql: &str) -> Vec<RecordBatch> {
.await
.unwrap();
let Output::Stream(stream) = engine
.execute(&plan)
.execute(plan, QueryContext::arc())
.await
.unwrap() else { unreachable!() };
util::collect(stream).await.unwrap()

View File

@@ -77,7 +77,7 @@ async fn test_datafusion_query_engine() -> Result<()> {
.unwrap(),
);
let output = engine.execute(&plan).await?;
let output = engine.execute(plan, QueryContext::arc()).await?;
let recordbatch = match output {
Output::Stream(recordbatch) => recordbatch,

View File

@@ -282,7 +282,7 @@ impl Script for PyScript {
.planner()
.plan(stmt, QueryContext::arc())
.await?;
let res = self.query_engine.execute(&plan).await?;
let res = self.query_engine.execute(plan, QueryContext::arc()).await?;
let copr = self.copr.clone();
match res {
Output::Stream(stream) => Ok(Output::Stream(Box::pin(CoprStream::try_new(
@@ -380,7 +380,7 @@ import greptime as gt
@copr(args=["number"], returns = ["number"], sql = "select * from numbers")
def test(number) -> vector[u32]:
from greptime import query
return query().sql("select * from numbers")[0][0]
return query().sql("select * from numbers")[0]
"#;
let script = script_engine
.compile(script, CompileContext::default())

View File

@@ -16,6 +16,6 @@ pub(crate) mod copr;
pub(crate) mod utils;
pub(crate) mod vector;
pub(crate) use copr::{check_args_anno_real_type, select_from_rb, Coprocessor};
pub(crate) use vector::PyVector;
pub(crate) use vector::{PyVector, PyVectorRef};
#[cfg(test)]
mod pair_tests;

View File

@@ -35,6 +35,7 @@ use rustpython_compiler_core::CodeObject;
use rustpython_vm as vm;
#[cfg(test)]
use serde::Deserialize;
use session::context::QueryContext;
use snafu::{OptionExt, ResultExt};
use vm::builtins::{PyList, PyListRef};
use vm::convert::ToPyObject;
@@ -260,11 +261,12 @@ pub(crate) fn check_args_anno_real_type(
.unwrap_or(true),
OtherSnafu {
reason: format!(
"column {}'s Type annotation is {:?}, but actual type is {:?}",
"column {}'s Type annotation is {:?}, but actual type is {:?} with nullable=={}",
// It's safe to unwrap here, we already ensure the args and types number is the same when parsing
copr.deco_args.arg_names.as_ref().unwrap()[idx],
anno_ty,
real_ty
real_ty,
is_nullable
)
}
)
@@ -343,20 +345,33 @@ pub(crate) enum Either {
Rb(RecordBatches),
AffectedRows(usize),
}
impl PyQueryEngine {
pub(crate) fn sql_to_rb(&self, sql: String) -> StdResult<RecordBatch, String> {
let res = self.query_with_new_thread(sql.clone())?;
match res {
Either::Rb(rbs) => {
let rb = compute::concat_batches(
rbs.schema().arrow_schema(),
rbs.iter().map(|r| r.df_record_batch()),
)
.map_err(|e| format!("Concat batches failed for query {sql}: {e}"))?;
RecordBatch::try_from_df_record_batch(rbs.schema(), rb)
.map_err(|e| format!("Convert datafusion record batch to record batch failed for query {sql}: {e}"))
}
Either::AffectedRows(_) => Err(format!("Expect actual results from query {sql}")),
}
}
}
#[rspyclass]
impl PyQueryEngine {
pub(crate) fn from_weakref(inner: QueryEngineWeakRef) -> Self {
Self { inner }
}
#[cfg(feature = "pyo3_backend")]
pub(crate) fn get_ref(&self) -> Option<Arc<dyn QueryEngine>> {
self.inner.0.upgrade()
}
pub(crate) fn query_with_new_thread(
&self,
query: Option<Arc<dyn QueryEngine>>,
s: String,
) -> StdResult<Either, String> {
pub(crate) fn query_with_new_thread(&self, s: String) -> StdResult<Either, String> {
let query = self.inner.0.upgrade();
let thread_handle = std::thread::spawn(move || -> std::result::Result<_, String> {
if let Some(engine) = query {
let stmt = QueryLanguageParser::parse_sql(&s).map_err(|e| e.to_string())?;
@@ -373,7 +388,7 @@ impl PyQueryEngine {
.map_err(|e| e.to_string())?;
let res = engine
.clone()
.execute(&plan)
.execute(plan, QueryContext::arc())
.await
.map_err(|e| e.to_string());
match res {
@@ -397,31 +412,36 @@ impl PyQueryEngine {
.map_err(|e| format!("Dedicated thread for sql query panic: {e:?}"))?
}
// TODO(discord9): find a better way to call sql query api, now we don't if we are in async context or not
/// return sql query results in List[List[PyVector]], or List[usize] for AffectedRows number if no recordbatches is returned
/// return sql query results in List[PyVector], or List[usize] for AffectedRows number if no recordbatches is returned
#[pymethod]
fn sql(&self, s: String, vm: &VirtualMachine) -> PyResult<PyListRef> {
let query = self.inner.0.upgrade();
self.query_with_new_thread(query, s)
self.query_with_new_thread(s)
.map_err(|e| vm.new_system_error(e))
.map(|rbs| match rbs {
Either::Rb(rbs) => {
let mut top_vec = Vec::with_capacity(rbs.iter().count());
for rb in rbs.iter() {
let mut vec_of_vec = Vec::with_capacity(rb.columns().len());
for v in rb.columns() {
let v = PyVector::from(v.clone());
vec_of_vec.push(v.to_pyobject(vm));
}
let vec_of_vec = PyList::new_ref(vec_of_vec, vm.as_ref()).to_pyobject(vm);
top_vec.push(vec_of_vec);
}
let top_vec = PyList::new_ref(top_vec, vm.as_ref());
top_vec
let rb = compute::concat_batches(
rbs.schema().arrow_schema(),
rbs.iter().map(|rb| rb.df_record_batch()),
)
.map_err(|e| {
vm.new_runtime_error(format!("Failed to concat batches: {e:#?}"))
})?;
let rb =
RecordBatch::try_from_df_record_batch(rbs.schema(), rb).map_err(|e| {
vm.new_runtime_error(format!("Failed to cast recordbatch: {e:#?}"))
})?;
let columns_vectors = rb
.columns()
.iter()
.map(|v| PyVector::from(v.clone()).to_pyobject(vm))
.collect::<Vec<_>>();
Ok(PyList::new_ref(columns_vectors, vm.as_ref()))
}
Either::AffectedRows(cnt) => {
PyList::new_ref(vec![vm.ctx.new_int(cnt).into()], vm.as_ref())
}
})
Either::AffectedRows(cnt) => Ok(PyList::new_ref(
vec![vm.ctx.new_int(cnt).into()],
vm.as_ref(),
)),
})?
}
}

View File

@@ -109,10 +109,12 @@ async fn integrated_py_copr_test() {
}
}
#[allow(clippy::print_stdout)]
#[test]
fn pyo3_rspy_test_in_pairs() {
let testcases = sample_test_case();
for case in testcases {
println!("Testcase: {}", case.script);
eval_rspy(case.clone());
#[cfg(feature = "pyo3_backend")]
eval_pyo3(case);
@@ -122,6 +124,9 @@ fn pyo3_rspy_test_in_pairs() {
fn check_equal(v0: VectorRef, v1: VectorRef) -> bool {
let v0 = v0.to_arrow_array();
let v1 = v1.to_arrow_array();
if v0.len() != v1.len() {
return false;
}
fn is_float(ty: &ArrowDataType) -> bool {
use ArrowDataType::*;
matches!(ty, Float16 | Float32 | Float64)

View File

@@ -17,10 +17,11 @@ use std::f64::consts;
use std::sync::Arc;
use datatypes::prelude::ScalarVector;
#[cfg(feature = "pyo3_backend")]
use datatypes::vectors::UInt32Vector;
use datatypes::vectors::{BooleanVector, Float64Vector, Int32Vector, Int64Vector, VectorRef};
use super::CoprTestCase;
use crate::python::ffi_types::pair_tests::CodeBlockTestCase;
use crate::python::ffi_types::pair_tests::{CodeBlockTestCase, CoprTestCase};
macro_rules! vector {
($ty: ident, $slice: expr) => {
Arc::new($ty::from_slice($slice)) as VectorRef
@@ -56,14 +57,14 @@ def boolean_array() -> vector[f64]:
from greptime import vector
from greptime import query, dataframe
try:
print("query()=", query())
except KeyError as e:
print("query()=", e)
print("query()=", query())
assert "query_engine object at" in str(query())
try:
print("dataframe()=", dataframe())
except KeyError as e:
print("dataframe()=", e)
print(str(e), type(str(e)), 'No __dataframe__' in str(e))
assert 'No __dataframe__' in str(e)
v = vector([1.0, 2.0, 3.0])
# This returns a vector([2.0])
@@ -72,6 +73,57 @@ def boolean_array() -> vector[f64]:
.to_string(),
expect: Some(ronish!("value": vector!(Float64Vector, [2.0f64]))),
},
CoprTestCase {
script: r#"
@copr(returns=["value"], backend="rspy")
def boolean_array() -> vector[f64]:
from greptime import vector, col
from greptime import query, dataframe, PyDataFrame
df = PyDataFrame.from_sql("select number from numbers limit 5")
print("df from sql=", df)
collected = df.collect()
print("df.collect()=", collected)
assert len(collected[0][0]) == 5
df = PyDataFrame.from_sql("select number from numbers limit 5").filter(col("number") > 2)
collected = df.collect()
assert len(collected[0][0]) == 2
print("query()=", query())
assert "query_engine object at" in repr(query())
try:
print("dataframe()=", dataframe())
except KeyError as e:
print("dataframe()=", e)
assert "__dataframe__" in str(e)
v = vector([1.0, 2.0, 3.0])
# This returns a vector([2.0])
return v[(v > 1) & (v < 3)]
"#
.to_string(),
expect: Some(ronish!("value": vector!(Float64Vector, [2.0f64]))),
},
#[cfg(feature = "pyo3_backend")]
CoprTestCase {
script: r#"
@copr(returns=["value"], backend="pyo3")
def boolean_array() -> vector[f64]:
from greptime import vector
from greptime import query, dataframe, PyDataFrame, col
df = PyDataFrame.from_sql("select number from numbers limit 5")
print("df from sql=", df)
ret = df.collect()
print("df.collect()=", ret)
assert len(ret[0][0]) == 5
df = PyDataFrame.from_sql("select number from numbers limit 5").filter(col("number") > 2)
collected = df.collect()
assert len(collected[0][0]) == 2
return ret[0][0]
"#
.to_string(),
expect: Some(ronish!("value": vector!(UInt32Vector, [0, 1, 2, 3, 4]))),
},
#[cfg(feature = "pyo3_backend")]
CoprTestCase {
script: r#"
@@ -178,6 +230,64 @@ def answer() -> vector[i64]:
.to_string(),
expect: Some(ronish!("number": vector!(Int64Vector, [1, 2]))),
},
#[cfg(feature = "pyo3_backend")]
CoprTestCase {
script: r#"
@copr(returns=["value"], backend="pyo3")
def answer() -> vector[i64]:
from greptime import vector
import pyarrow as pa
a = vector.from_pyarrow(pa.array([42, 43, 44]))
return a[0:1]
"#
.to_string(),
expect: Some(ronish!("value": vector!(Int64Vector, [42]))),
},
#[cfg(feature = "pyo3_backend")]
CoprTestCase {
script: r#"
@copr(returns=["value"], backend="pyo3")
def answer() -> vector[i64]:
from greptime import vector
a = vector([42, 43, 44])
# slicing test
assert a[0:2] == a[:-1]
assert len(a[:-1]) == vector([42,44])
assert a[0:1] == a[:-2]
assert a[0:1] == vector([42])
assert a[:-2] == vector([42])
assert a[:-1:2] == vector([42])
assert a[::2] == vector([42,44])
# negative step
assert a[-1::-2] == vector([44, 42])
assert a[-2::-2] == vector([44])
return a[0:1]
"#
.to_string(),
expect: Some(ronish!("value": vector!(Int64Vector, [42]))),
},
CoprTestCase {
script: r#"
@copr(returns=["value"], backend="rspy")
def answer() -> vector[i64]:
from greptime import vector
a = vector([42, 43, 44])
# slicing test
assert a[0:2] == a[:-1]
assert len(a[:-1]) == vector([42,44])
assert a[0:1] == a[:-2]
assert a[0:1] == vector([42])
assert a[:-2] == vector([42])
assert a[:-1:2] == vector([42])
assert a[::2] == vector([42,44])
# negative step
assert a[-1::-2] == vector([44, 42])
assert a[-2::-2] == vector([44])
return a[-2:-1]
"#
.to_string(),
expect: Some(ronish!("value": vector!(Int64Vector, [43]))),
},
]
}
@@ -185,6 +295,7 @@ def answer() -> vector[i64]:
/// Using a function to generate testcase instead of `.ron` configure file because it's more flexible and we are in #[cfg(test)] so no binary bloat worrying
#[allow(clippy::approx_constant)]
pub(super) fn sample_test_case() -> Vec<CodeBlockTestCase> {
// TODO(discord9): detailed tests for slicing vector
vec![
CodeBlockTestCase {
input: ronish! {
@@ -192,13 +303,54 @@ pub(super) fn sample_test_case() -> Vec<CodeBlockTestCase> {
},
script: r#"
from greptime import *
ret = a+3.0
ret = ret * 2.0
ret = ret / 2.0
ret = ret - 3.0
ret = a[0:1]
ret"#
.to_string(),
expect: vector!(Float64Vector, [1.0f64, 2.0, 3.0]),
expect: vector!(Float64Vector, [1.0f64]),
},
CodeBlockTestCase {
input: ronish! {
"a": vector!(Float64Vector, [1.0f64, 2.0, 3.0])
},
script: r#"
from greptime import *
ret = a[0:1:1]
ret"#
.to_string(),
expect: vector!(Float64Vector, [1.0f64]),
},
CodeBlockTestCase {
input: ronish! {
"a": vector!(Float64Vector, [1.0f64, 2.0, 3.0])
},
script: r#"
from greptime import *
ret = a[-2:-1]
ret"#
.to_string(),
expect: vector!(Float64Vector, [2.0f64]),
},
CodeBlockTestCase {
input: ronish! {
"a": vector!(Float64Vector, [1.0f64, 2.0, 3.0])
},
script: r#"
from greptime import *
ret = a[-1:-2:-1]
ret"#
.to_string(),
expect: vector!(Float64Vector, [3.0f64]),
},
CodeBlockTestCase {
input: ronish! {
"a": vector!(Float64Vector, [1.0f64, 2.0, 3.0])
},
script: r#"
from greptime import *
ret = a[-1:-4:-1]
ret"#
.to_string(),
expect: vector!(Float64Vector, [3.0f64, 2.0, 1.0]),
},
CodeBlockTestCase {
input: ronish! {

View File

@@ -49,6 +49,8 @@ pub struct PyVector {
pub(crate) vector: VectorRef,
}
pub(crate) type PyVectorRef = PyRef<PyVector>;
impl From<VectorRef> for PyVector {
fn from(vector: VectorRef) -> Self {
Self { vector }
@@ -381,7 +383,6 @@ impl PyVector {
// adjust_indices so negative number is transform to usize
let (mut range, step, slice_len) = slice.adjust_indices(self.len());
let vector = self.as_vector_ref();
let mut buf = vector.data_type().create_mutable_vector(slice_len);
if slice_len == 0 {
let v: PyVector = buf.to_vector().into();
@@ -391,6 +392,7 @@ impl PyVector {
Ok(v.into_pyobject(vm))
} else if step.is_negative() {
// Negative step require special treatment
// range.start > range.stop if slice can found no-empty
for i in range.rev().step_by(step.unsigned_abs()) {
// Safety: This mutable vector is created from the vector's data type.
buf.push_value_ref(vector.get_ref(i));

View File

@@ -61,6 +61,7 @@ macro_rules! batch_import {
#[pyo3(name = "greptime")]
pub(crate) fn greptime_builtins(_py: Python<'_>, m: &PyModule) -> PyResult<()> {
m.add_class::<PyVector>()?;
m.add_class::<PyDataFrame>()?;
use self::query_engine;
batch_import!(
m,
@@ -137,7 +138,7 @@ fn dataframe(py: Python) -> PyResult<PyDataFrame> {
#[pyfunction]
#[pyo3(name = "query")]
fn query_engine(py: Python) -> PyResult<PyQueryEngine> {
pub(crate) fn query_engine(py: Python) -> PyResult<PyQueryEngine> {
let globals = get_globals(py)?;
let query = globals
.get_item("__query__")

View File

@@ -16,7 +16,7 @@ use std::collections::HashMap;
use common_recordbatch::RecordBatch;
use datatypes::vectors::{Helper, VectorRef};
use pyo3::exceptions::PyValueError;
use pyo3::exceptions::{PyRuntimeError, PyValueError};
use pyo3::types::{PyDict, PyList, PyModule, PyTuple};
use pyo3::{pymethods, PyAny, PyCell, PyObject, PyResult, Python, ToPyObject};
use snafu::{ensure, Backtrace, GenerateImplicitData, ResultExt};
@@ -31,9 +31,8 @@ use crate::python::pyo3::utils::{init_cpython_interpreter, pyo3_obj_try_to_typed
impl PyQueryEngine {
#[pyo3(name = "sql")]
pub(crate) fn sql_pyo3(&self, py: Python<'_>, s: String) -> PyResult<PyObject> {
let query = self.get_ref();
let res = self
.query_with_new_thread(query, s)
.query_with_new_thread(s)
.map_err(PyValueError::new_err)?;
match res {
crate::python::ffi_types::copr::Either::Rb(rbs) => {
@@ -143,7 +142,7 @@ coprocessor = copr
py_any_to_vec(result, col_len)
})()
.map_err(|err| error::Error::PyRuntime {
msg: err.to_string(),
msg: err.into_value(py).to_string(),
backtrace: Backtrace::generate(),
})?;
ensure!(
@@ -165,6 +164,23 @@ coprocessor = copr
/// Cast return of py script result to `Vec<VectorRef>`,
/// constants will be broadcast to length of `col_len`
fn py_any_to_vec(obj: &PyAny, col_len: usize) -> PyResult<Vec<VectorRef>> {
// check if obj is of two types:
// 1. tuples of PyVector
// 2. a single PyVector
let check = if obj.is_instance_of::<PyTuple>()? {
let tuple = obj.downcast::<PyTuple>()?;
(0..tuple.len())
.map(|idx| tuple.get_item(idx).map(|i| i.is_instance_of::<PyVector>()))
.all(|i| matches!(i, Ok(Ok(true))))
} else {
obj.is_instance_of::<PyVector>()?
};
if !check {
return Err(PyRuntimeError::new_err(format!(
"Expect a tuple of vectors or one single vector, found {obj}"
)));
}
if let Ok(tuple) = obj.downcast::<PyTuple>() {
let len = tuple.len();
let v = (0..len)
@@ -219,7 +235,7 @@ def a(cpu, mem, **kwargs):
for k, v in kwargs.items():
print("%s == %s" % (k, v))
print(dataframe().select([col("cpu")<lit(0.3)]).collect())
return (0.5 < cpu) & ~( cpu >= 0.75)
return (0.5 < cpu) & ~(cpu >= 0.75)
"#;
let cpu_array = Float32Vector::from_slice([0.9f32, 0.8, 0.7, 0.3]);
let mem_array = Float64Vector::from_slice([0.1f64, 0.2, 0.3, 0.4]);

View File

@@ -15,14 +15,15 @@
use common_recordbatch::DfRecordBatch;
use datafusion::dataframe::DataFrame as DfDataFrame;
use datafusion_expr::Expr as DfExpr;
use pyo3::exceptions::PyValueError;
use pyo3::exceptions::{PyRuntimeError, PyValueError};
use pyo3::prelude::*;
use pyo3::pyclass::CompareOp;
use pyo3::types::PyList;
use pyo3::types::{PyList, PyType};
use snafu::ResultExt;
use crate::python::error::DataFusionSnafu;
use crate::python::ffi_types::PyVector;
use crate::python::pyo3::builtins::query_engine;
use crate::python::pyo3::utils::pyo3_obj_try_to_typed_scalar_value;
use crate::python::utils::block_on_async;
type PyExprRef = Py<PyExpr>;
@@ -49,6 +50,15 @@ impl PyDataFrame {
#[pymethods]
impl PyDataFrame {
#[classmethod]
fn from_sql(_cls: &PyType, py: Python, sql: String) -> PyResult<Self> {
let query = query_engine(py)?;
let rb = query.sql_to_rb(sql).map_err(PyRuntimeError::new_err)?;
let ctx = datafusion::execution::context::SessionContext::new();
ctx.read_batch(rb.df_record_batch().clone())
.map_err(|e| PyRuntimeError::new_err(format!("{e:?}")))
.map(Self::from)
}
fn __call__(&self) -> PyResult<Self> {
Ok(self.clone())
}

View File

@@ -24,7 +24,7 @@ use datatypes::vectors::Helper;
use pyo3::exceptions::{PyIndexError, PyRuntimeError, PyValueError};
use pyo3::prelude::*;
use pyo3::pyclass::CompareOp;
use pyo3::types::{PyBool, PyFloat, PyInt, PyList, PyString, PyType};
use pyo3::types::{PyBool, PyFloat, PyInt, PyList, PySlice, PyString, PyType};
use super::utils::val_to_py_any;
use crate::python::ffi_types::vector::{arrow_rtruediv, wrap_bool_result, wrap_result, PyVector};
@@ -236,6 +236,24 @@ impl PyVector {
fn __invert__(&self) -> PyResult<Self> {
Self::vector_invert(self).map_err(PyValueError::new_err)
}
#[pyo3(name = "concat")]
fn pyo3_concat(&self, py: Python<'_>, other: &Self) -> PyResult<Self> {
py.allow_threads(|| {
let left = self.to_arrow_array();
let right = other.to_arrow_array();
let res = compute::concat(&[left.as_ref(), right.as_ref()]);
let res = res.map_err(|err| PyValueError::new_err(format!("Arrow Error: {err:#?}")))?;
let ret = Helper::try_into_vector(res.clone()).map_err(|e| {
PyValueError::new_err(format!(
"Can't cast result into vector, result: {res:?}, err: {e:?}",
))
})?;
Ok(ret.into())
})
}
/// take a boolean array and filters the Array, returning elements matching the filter (i.e. where the values are true).
#[pyo3(name = "filter")]
fn pyo3_filter(&self, py: Python<'_>, other: &Self) -> PyResult<Self> {
@@ -299,6 +317,49 @@ impl PyVector {
})?;
let ret = Self::from(ret).into_py(py);
Ok(ret)
} else if let Ok(slice) = needle.downcast::<PySlice>(py) {
let indices = slice.indices(self.len() as i64)?;
let (start, stop, step, _slicelength) = (
indices.start,
indices.stop,
indices.step,
indices.slicelength,
);
if start < 0 {
return Err(PyValueError::new_err(format!(
"Negative start is not supported, found {start} in {indices:?}"
)));
} // Negative stop is supported, means from "indices.start" to the actual start of the vector
let vector = self.as_vector_ref();
let mut buf = vector
.data_type()
.create_mutable_vector(indices.slicelength as usize);
let v = if indices.slicelength == 0 {
buf.to_vector()
} else {
if indices.step > 0 {
let range = if stop == -1 {
start as usize..start as usize
} else {
start as usize..stop as usize
};
for i in range.step_by(step.unsigned_abs()) {
buf.push_value_ref(vector.get_ref(i));
}
} else {
// if no-empty, then stop < start
// note: start..stop is empty is start >= stop
// stop>=-1
let range = { (stop + 1) as usize..=start as usize };
for i in range.rev().step_by(step.unsigned_abs()) {
buf.push_value_ref(vector.get_ref(i));
}
}
buf.to_vector()
};
let v: PyVector = v.into();
Ok(v.into_py(py))
} else if let Ok(index) = needle.extract::<isize>(py) {
// deal with negative index
let len = self.len() as isize;

View File

@@ -290,6 +290,7 @@ pub(crate) mod greptime_builtin {
use common_function::scalars::math::PowFunction;
use common_function::scalars::{Function, FunctionRef, FUNCTION_REGISTRY};
use datafusion::arrow::datatypes::DataType as ArrowDataType;
use datafusion::dataframe::DataFrame as DfDataFrame;
use datafusion::physical_plan::expressions;
use datafusion_expr::{ColumnarValue as DFColValue, Expr as DfExpr};
use datafusion_physical_expr::math_expressions;
@@ -300,19 +301,26 @@ pub(crate) mod greptime_builtin {
use paste::paste;
use rustpython_vm::builtins::{PyFloat, PyFunction, PyInt, PyStr};
use rustpython_vm::function::{FuncArgs, KwArgs, OptionalArg};
use rustpython_vm::{AsObject, PyObjectRef, PyPayload, PyRef, PyResult, VirtualMachine};
use rustpython_vm::{
pyclass, AsObject, PyObjectRef, PyPayload, PyRef, PyResult, VirtualMachine,
};
use super::{
use crate::python::ffi_types::copr::PyQueryEngine;
use crate::python::ffi_types::vector::val_to_pyobj;
use crate::python::ffi_types::{PyVector, PyVectorRef};
use crate::python::rspython::builtins::{
all_to_f64, eval_aggr_fn, from_df_err, try_into_columnar_value, try_into_py_obj,
type_cast_error,
};
use crate::python::ffi_types::copr::PyQueryEngine;
use crate::python::ffi_types::vector::val_to_pyobj;
use crate::python::ffi_types::PyVector;
use crate::python::rspython::dataframe_impl::data_frame::{PyDataFrame, PyExpr, PyExprRef};
use crate::python::rspython::utils::{
is_instance, py_obj_to_value, py_obj_to_vec, PyVectorRef,
};
use crate::python::rspython::dataframe_impl::data_frame::{PyExpr, PyExprRef};
use crate::python::rspython::utils::{is_instance, py_obj_to_value, py_obj_to_vec};
#[pyattr]
#[pyclass(module = "greptime_builtin", name = "PyDataFrame")]
#[derive(PyPayload, Debug, Clone)]
pub struct PyDataFrame {
pub inner: DfDataFrame,
}
/// get `__dataframe__` from globals and return it
/// TODO(discord9): this is a terrible hack, we should find a better way to get `__dataframe__`
@@ -327,9 +335,9 @@ pub(crate) mod greptime_builtin {
}
/// get `__query__` from globals and return it
/// TODO(discord9): this is a terrible hack, we should find a better way to get `__dataframe__`
/// TODO(discord9): this is a terrible hack, we should find a better way to get `__query__`
#[pyfunction]
fn query(vm: &VirtualMachine) -> PyResult<PyQueryEngine> {
pub(crate) fn query(vm: &VirtualMachine) -> PyResult<PyQueryEngine> {
let query_engine = vm.current_globals().get_item("__query__", vm)?;
let query_engine = query_engine.payload::<PyQueryEngine>().ok_or_else(|| {
vm.new_type_error(format!("object {:?} is not a QueryEngine", query_engine))

View File

@@ -14,8 +14,10 @@
use rustpython_vm::class::PyClassImpl;
use rustpython_vm::{pymodule as rspymodule, VirtualMachine};
use crate::python::rspython::builtins::greptime_builtin::PyDataFrame;
pub(crate) fn init_data_frame(module_name: &str, vm: &mut VirtualMachine) {
data_frame::PyDataFrame::make_class(&vm.ctx);
PyDataFrame::make_class(&vm.ctx);
data_frame::PyExpr::make_class(&vm.ctx);
vm.add_native_module(module_name.to_owned(), Box::new(data_frame::make_module));
}
@@ -24,6 +26,7 @@ pub(crate) fn init_data_frame(module_name: &str, vm: &mut VirtualMachine) {
pub(crate) mod data_frame {
use common_recordbatch::{DfRecordBatch, RecordBatch};
use datafusion::dataframe::DataFrame as DfDataFrame;
use datafusion::execution::context::SessionContext;
use datafusion_expr::Expr as DfExpr;
use rustpython_vm::builtins::{PyList, PyListRef};
use rustpython_vm::function::PyComparisonValue;
@@ -35,13 +38,10 @@ pub(crate) mod data_frame {
use crate::python::error::DataFusionSnafu;
use crate::python::ffi_types::PyVector;
use crate::python::rspython::builtins::greptime_builtin::lit;
use crate::python::rspython::builtins::greptime_builtin::{
lit, query as get_query_engine, PyDataFrame,
};
use crate::python::utils::block_on_async;
#[rspyclass(module = "data_frame", name = "DataFrame")]
#[derive(PyPayload, Debug, Clone)]
pub struct PyDataFrame {
pub inner: DfDataFrame,
}
impl From<DfDataFrame> for PyDataFrame {
fn from(inner: DfDataFrame) -> Self {
@@ -63,9 +63,20 @@ pub(crate) mod data_frame {
}
#[rspyclass]
impl PyDataFrame {
#[pymethod]
fn from_sql(sql: String, vm: &VirtualMachine) -> PyResult<Self> {
let query_engine = get_query_engine(vm)?;
let rb = query_engine.sql_to_rb(sql.clone()).map_err(|e| {
vm.new_runtime_error(format!("failed to execute sql: {:?}, error: {:?}", sql, e))
})?;
let ctx = SessionContext::new();
ctx.read_batch(rb.df_record_batch().clone())
.map_err(|e| vm.new_runtime_error(format!("{e:?}")))
.map(|df| df.into())
}
/// TODO(discord9): error handling
fn from_record_batch(rb: &DfRecordBatch) -> crate::python::error::Result<Self> {
let ctx = datafusion::execution::context::SessionContext::new();
let ctx = SessionContext::new();
let inner = ctx.read_batch(rb.clone()).context(DataFusionSnafu)?;
Ok(Self { inner })
}

View File

@@ -22,7 +22,7 @@ use datatypes::vectors::{
BooleanVector, Float64Vector, Helper, Int64Vector, NullVector, StringVector, VectorRef,
};
use rustpython_vm::builtins::{PyBaseExceptionRef, PyBool, PyFloat, PyInt, PyList, PyStr};
use rustpython_vm::{PyObjectRef, PyPayload, PyRef, PyResult, VirtualMachine};
use rustpython_vm::{PyObjectRef, PyPayload, PyResult, VirtualMachine};
use snafu::{Backtrace, GenerateImplicitData, OptionExt, ResultExt};
use crate::python::error;
@@ -30,8 +30,6 @@ use crate::python::error::ret_other_error_with;
use crate::python::ffi_types::PyVector;
use crate::python::rspython::builtins::try_into_columnar_value;
pub(crate) type PyVectorRef = PyRef<PyVector>;
/// use `rustpython`'s `is_instance` method to check if a PyObject is a instance of class.
/// if `PyResult` is Err, then this function return `false`
pub fn is_instance<T: PyPayload>(obj: &PyObjectRef, vm: &VirtualMachine) -> bool {

View File

@@ -167,7 +167,7 @@ impl ScriptsTable {
let stream = match self
.query_engine
.execute(&plan)
.execute(plan, QueryContext::arc())
.await
.context(FindScriptSnafu { name })?
{

View File

@@ -18,8 +18,7 @@ use api::v1::ddl_request::Expr;
use api::v1::greptime_request::Request;
use api::v1::{DdlRequest, FlushTableExpr};
use axum::extract::{Query, RawBody, State};
use axum::http::StatusCode as HttpStatusCode;
use axum::Json;
use axum::http::StatusCode;
use session::context::QueryContext;
use snafu::OptionExt;
@@ -32,21 +31,20 @@ pub async fn flush(
State(grpc_handler): State<ServerGrpcQueryHandlerRef>,
Query(params): Query<HashMap<String, String>>,
RawBody(_): RawBody,
) -> Result<(HttpStatusCode, Json<String>)> {
) -> Result<(StatusCode, ())> {
let catalog_name = params
.get("catalog_name")
.get("catalog")
.cloned()
.unwrap_or("greptime".to_string());
let schema_name =
params
.get("schema_name")
.cloned()
.context(error::InvalidFlushArgumentSnafu {
err_msg: "schema_name is not present",
})?;
let schema_name = params
.get("db")
.cloned()
.context(error::InvalidFlushArgumentSnafu {
err_msg: "db is not present",
})?;
// if table name is not present, flush all tables inside schema
let table_name = params.get("table_name").cloned().unwrap_or_default();
let table_name = params.get("table").cloned().unwrap_or_default();
let region_id: Option<u32> = params
.get("region")
@@ -65,5 +63,5 @@ pub async fn flush(
});
grpc_handler.do_query(request, QueryContext::arc()).await?;
Ok((HttpStatusCode::OK, Json::from("hello, world".to_string())))
Ok((StatusCode::NO_CONTENT, ()))
}

View File

@@ -72,10 +72,10 @@ impl SqlQueryHandler for DummyInstance {
let plan = self
.query_engine
.planner()
.plan(stmt, query_ctx)
.plan(stmt, query_ctx.clone())
.await
.unwrap();
let output = self.query_engine.execute(&plan).await.unwrap();
let output = self.query_engine.execute(plan, query_ctx).await.unwrap();
vec![Ok(output)]
}

View File

@@ -31,7 +31,7 @@ impl<'a> ParserContext<'a> {
match spstatement {
SpStatement::Delete { .. } => {
Ok(Statement::Delete(Box::new(Delete::try_from(spstatement)?)))
Ok(Statement::Delete(Box::new(Delete { inner: spstatement })))
}
unexp => error::UnsupportedSnafu {
sql: self.sql.to_string(),

View File

@@ -12,58 +12,9 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use sqlparser::ast::{Expr, ObjectName, Statement, TableFactor};
use crate::error::{Error, InvalidSqlSnafu, Result};
use sqlparser::ast::Statement;
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct Delete {
table_name: ObjectName,
selection: Option<Expr>,
}
impl Delete {
pub fn table_name(&self) -> &ObjectName {
&self.table_name
}
pub fn selection(&self) -> &Option<Expr> {
&self.selection
}
}
impl TryFrom<Statement> for Delete {
type Error = Error;
fn try_from(stmt: Statement) -> Result<Self> {
match stmt {
Statement::Delete {
table_name,
using,
selection,
returning,
} => {
if using.is_some() || returning.is_some() {
return InvalidSqlSnafu {
msg: "delete sql isn't support using and returning.".to_string(),
}
.fail();
}
match table_name {
TableFactor::Table { name, .. } => Ok(Delete {
table_name: name,
selection,
}),
_ => InvalidSqlSnafu {
msg: "can't find table name, tableFactor is not Table type".to_string(),
}
.fail(),
}
}
unexp => InvalidSqlSnafu {
msg: format!("Not expected to be {unexp}"),
}
.fail(),
}
}
pub inner: Statement,
}

View File

@@ -79,6 +79,8 @@ impl TryFrom<&Statement> for DfStatement {
let s = match s {
Statement::Query(query) => SpStatement::Query(Box::new(query.inner.clone())),
Statement::Explain(explain) => explain.inner.clone(),
Statement::Insert(insert) => insert.inner.clone(),
Statement::Delete(delete) => delete.inner.clone(),
_ => {
return ConvertToDfStatementSnafu {
statement: format!("{s:?}"),

View File

@@ -187,7 +187,7 @@ impl ChunkReaderBuilder {
);
continue;
}
let reader = self.sst_layer.read_sst(file.file_id(), &read_opts).await?;
let reader = self.sst_layer.read_sst(file.clone(), &read_opts).await?;
reader_builder = reader_builder.push_batch_reader(reader);
}

View File

@@ -143,12 +143,14 @@ fn infer_time_bucket(files: &[FileHandle]) -> i64 {
}
/// A set of predefined time buckets.
const TIME_BUCKETS: [i64; 5] = [
60 * 60, // one hour
2 * 60 * 60, // two hours
12 * 60 * 60, // twelve hours
24 * 60 * 60, // one day
7 * 24 * 60 * 60, // one week
const TIME_BUCKETS: [i64; 7] = [
60 * 60, // one hour
2 * 60 * 60, // two hours
12 * 60 * 60, // twelve hours
24 * 60 * 60, // one day
7 * 24 * 60 * 60, // one week
365 * 24 * 60 * 60, // one year
10 * 365 * 24 * 60 * 60, // ten years
];
/// Fits a given time span into time bucket by find the minimum bucket that can cover the span.
@@ -213,7 +215,7 @@ mod tests {
assert_eq!(TIME_BUCKETS[2], fit_time_bucket(TIME_BUCKETS[2] - 1));
assert_eq!(TIME_BUCKETS[2], fit_time_bucket(TIME_BUCKETS[2]));
assert_eq!(TIME_BUCKETS[3], fit_time_bucket(TIME_BUCKETS[3] - 1));
assert_eq!(TIME_BUCKETS[4], fit_time_bucket(i64::MAX));
assert_eq!(TIME_BUCKETS[6], fit_time_bucket(i64::MAX));
}
#[test]

View File

@@ -80,10 +80,11 @@ impl<S: LogStore> CompactionTaskImpl<S> {
});
}
let outputs = futures::future::join_all(futs)
.await
let outputs = futures::future::try_join_all(futs)
.await?
.into_iter()
.collect::<Result<_>>()?;
.flatten()
.collect();
let inputs = compacted_inputs.into_iter().collect();
Ok((outputs, inputs))
}
@@ -162,7 +163,7 @@ impl CompactionOutput {
region_id: RegionId,
schema: RegionSchemaRef,
sst_layer: AccessLayerRef,
) -> Result<FileMeta> {
) -> Result<Option<FileMeta>> {
let reader = build_sst_reader(
schema,
sst_layer.clone(),
@@ -175,20 +176,21 @@ impl CompactionOutput {
let output_file_id = FileId::random();
let opts = WriteOptions {};
let SstInfo {
time_range,
file_size,
} = sst_layer
Ok(sst_layer
.write_sst(output_file_id, Source::Reader(reader), &opts)
.await?;
Ok(FileMeta {
region_id,
file_id: output_file_id,
time_range,
level: self.output_level,
file_size,
})
.await?
.map(
|SstInfo {
time_range,
file_size,
}| FileMeta {
region_id,
file_id: output_file_id,
time_range,
level: self.output_level,
file_size,
},
))
}
}

View File

@@ -227,6 +227,7 @@ mod tests {
} = writer
.write_sst(&sst::WriteOptions::default())
.await
.unwrap()
.unwrap();
let handle = FileHandle::new(
FileMeta {
@@ -415,6 +416,7 @@ mod tests {
)
.write_sst(&opts)
.await
.unwrap()
.unwrap();
assert_eq!(
Some((
@@ -431,6 +433,7 @@ mod tests {
)
.write_sst(&opts)
.await
.unwrap()
.unwrap();
assert_eq!(
Some((
@@ -447,6 +450,7 @@ mod tests {
)
.write_sst(&opts)
.await
.unwrap()
.unwrap();
assert_eq!(

View File

@@ -145,6 +145,7 @@ mod tests {
let sst_info = layer
.write_sst(sst_file_id, Source::Iter(iter), &WriteOptions {})
.await
.unwrap()
.unwrap();
(

View File

@@ -202,28 +202,28 @@ impl<S: LogStore> FlushJob<S> {
let sst_layer = self.sst_layer.clone();
futures.push(async move {
let SstInfo {
time_range,
file_size,
} = sst_layer
Ok(sst_layer
.write_sst(file_id, Source::Iter(iter), &WriteOptions::default())
.await?;
Ok(FileMeta {
region_id,
file_id,
time_range,
level: 0,
file_size,
})
.await?
.map(
|SstInfo {
time_range,
file_size,
}| FileMeta {
region_id,
file_id,
time_range,
level: 0,
file_size,
},
))
});
}
let metas = futures_util::future::join_all(futures)
.await
.into_iter()
.collect::<Result<Vec<_>>>()?
let metas = futures_util::future::try_join_all(futures)
.await?
.into_iter()
.flatten()
.collect();
logging::info!("Successfully flush memtables to files: {:?}", metas);

View File

@@ -203,6 +203,13 @@ impl FileHandle {
self.inner.meta.file_id.as_parquet()
}
#[inline]
pub fn file_path(&self) -> String {
self.inner
.sst_layer
.sst_file_path(&self.inner.meta.file_id.as_parquet())
}
#[inline]
pub fn file_id(&self) -> FileId {
self.inner.meta.file_id
@@ -276,7 +283,7 @@ impl Drop for FileHandleInner {
);
}
Err(e) => {
error!(e; "Failed to schedule SST purge task, region: {}, name: {}",
error!(e; "Failed to schedule SST purge task, region: {}, name: {}",
self.meta.region_id, self.meta.file_id.as_parquet());
}
}
@@ -392,16 +399,24 @@ pub struct SstInfo {
/// SST access layer.
#[async_trait]
pub trait AccessLayer: Send + Sync + std::fmt::Debug {
/// Writes SST file with given `file_name`.
/// Returns the sst file path.
fn sst_file_path(&self, file_name: &str) -> String;
/// Writes SST file with given `file_id` and returns the SST info.
/// If source does not contain any data, `write_sst` will return `Ok(None)`.
async fn write_sst(
&self,
file_id: FileId,
source: Source,
opts: &WriteOptions,
) -> Result<SstInfo>;
) -> Result<Option<SstInfo>>;
/// Read SST file with given `file_name` and schema.
async fn read_sst(&self, file_id: FileId, opts: &ReadOptions) -> Result<BoxedBatchReader>;
/// Read SST file with given `file_handle` and schema.
async fn read_sst(
&self,
file_handle: FileHandle,
opts: &ReadOptions,
) -> Result<BoxedBatchReader>;
/// Deletes a SST file with given name.
async fn delete_sst(&self, file_id: FileId) -> Result<()>;
@@ -436,7 +451,7 @@ impl Source {
}
}
/// Sst access layer based on local file system.
/// Sst access layer.
#[derive(Debug)]
pub struct FsAccessLayer {
sst_dir: String,
@@ -450,21 +465,21 @@ impl FsAccessLayer {
object_store,
}
}
#[inline]
fn sst_file_path(&self, file_name: &str) -> String {
format!("{}{}", self.sst_dir, file_name)
}
}
#[async_trait]
impl AccessLayer for FsAccessLayer {
fn sst_file_path(&self, file_name: &str) -> String {
format!("{}{}", self.sst_dir, file_name)
}
/// Writes SST file with given `file_id`.
async fn write_sst(
&self,
file_id: FileId,
source: Source,
opts: &WriteOptions,
) -> Result<SstInfo> {
) -> Result<Option<SstInfo>> {
// Now we only supports parquet format. We may allow caller to specific SST format in
// WriteOptions in the future.
let file_path = self.sst_file_path(&file_id.as_parquet());
@@ -472,10 +487,14 @@ impl AccessLayer for FsAccessLayer {
writer.write_sst(opts).await
}
async fn read_sst(&self, file_id: FileId, opts: &ReadOptions) -> Result<BoxedBatchReader> {
let file_path = self.sst_file_path(&file_id.as_parquet());
/// Read SST file with given `file_handle` and schema.
async fn read_sst(
&self,
file_handle: FileHandle,
opts: &ReadOptions,
) -> Result<BoxedBatchReader> {
let reader = ParquetReader::new(
&file_path,
file_handle,
self.object_store.clone(),
opts.projected_schema.clone(),
opts.predicate.clone(),
@@ -486,6 +505,7 @@ impl AccessLayer for FsAccessLayer {
Ok(Box::new(stream))
}
/// Deletes a SST file with given file id.
async fn delete_sst(&self, file_id: FileId) -> Result<()> {
let path = self.sst_file_path(&file_id.as_parquet());
let object = self.object_store.object(&path);

View File

@@ -56,7 +56,7 @@ use crate::read::{Batch, BatchReader};
use crate::schema::compat::ReadAdapter;
use crate::schema::{ProjectedSchemaRef, StoreSchema, StoreSchemaRef};
use crate::sst;
use crate::sst::{Source, SstInfo};
use crate::sst::{FileHandle, Source, SstInfo};
/// Parquet sst writer.
pub struct ParquetWriter<'a> {
file_path: &'a str,
@@ -75,14 +75,17 @@ impl<'a> ParquetWriter<'a> {
}
}
pub async fn write_sst(self, _opts: &sst::WriteOptions) -> Result<SstInfo> {
pub async fn write_sst(self, _opts: &sst::WriteOptions) -> Result<Option<SstInfo>> {
self.write_rows(None).await
}
/// Iterates memtable and writes rows to Parquet file.
/// A chunk of records yielded from each iteration with a size given
/// in config will be written to a single row group.
async fn write_rows(mut self, extra_meta: Option<HashMap<String, String>>) -> Result<SstInfo> {
async fn write_rows(
mut self,
extra_meta: Option<HashMap<String, String>>,
) -> Result<Option<SstInfo>> {
let projected_schema = self.source.projected_schema();
let store_schema = projected_schema.schema_to_read();
let schema = store_schema.arrow_schema().clone();
@@ -106,6 +109,7 @@ impl<'a> ParquetWriter<'a> {
let mut arrow_writer = ArrowWriter::try_new(&mut buf, schema.clone(), Some(writer_props))
.context(WriteParquetSnafu)?;
let mut batches_written = 0;
while let Some(batch) = self.source.next_batch().await? {
let arrow_batch = RecordBatch::try_new(
schema.clone(),
@@ -119,8 +123,13 @@ impl<'a> ParquetWriter<'a> {
arrow_writer
.write(&arrow_batch)
.context(WriteParquetSnafu)?;
batches_written += 1;
}
if batches_written == 0 {
// if the source does not contain any batch, we skip writing an empty parquet file.
return Ok(None);
}
let file_meta = arrow_writer.close().context(WriteParquetSnafu)?;
let time_range = decode_timestamp_range(&file_meta, store_schema)
@@ -137,10 +146,10 @@ impl<'a> ParquetWriter<'a> {
path: object.path(),
})?
.content_length();
Ok(SstInfo {
Ok(Some(SstInfo {
time_range,
file_size,
})
}))
}
}
@@ -210,30 +219,38 @@ fn decode_timestamp_range_inner(
end = end.max(max);
}
assert!(
start <= end,
"Illegal timestamp range decoded from SST file {:?}, start: {}, end: {}",
file_meta,
start,
end
);
Ok(Some((
Timestamp::new(start, unit),
Timestamp::new(end, unit),
)))
}
pub struct ParquetReader<'a> {
file_path: &'a str,
pub struct ParquetReader {
// Holds the file handle to avoid the file purge purge it.
file_handle: FileHandle,
object_store: ObjectStore,
projected_schema: ProjectedSchemaRef,
predicate: Predicate,
time_range: TimestampRange,
}
impl<'a> ParquetReader<'a> {
impl ParquetReader {
pub fn new(
file_path: &str,
file_handle: FileHandle,
object_store: ObjectStore,
projected_schema: ProjectedSchemaRef,
predicate: Predicate,
time_range: TimestampRange,
) -> ParquetReader {
ParquetReader {
file_path,
file_handle,
object_store,
projected_schema,
predicate,
@@ -242,28 +259,24 @@ impl<'a> ParquetReader<'a> {
}
pub async fn chunk_stream(&self) -> Result<ChunkStream> {
let file_path = self.file_handle.file_path();
let operator = self.object_store.clone();
let reader = operator
.object(self.file_path)
.object(&file_path)
.reader()
.await
.context(ReadObjectSnafu {
path: self.file_path,
})?
.context(ReadObjectSnafu { path: &file_path })?
.compat();
let buf_reader = BufReader::new(reader);
let builder = ParquetRecordBatchStreamBuilder::new(buf_reader)
.await
.context(ReadParquetSnafu {
file: self.file_path,
})?;
.context(ReadParquetSnafu { file: &file_path })?;
let arrow_schema = builder.schema().clone();
let store_schema = Arc::new(StoreSchema::try_from(arrow_schema).context(
error::ConvertStoreSchemaSnafu {
file: self.file_path,
},
)?);
let store_schema = Arc::new(
StoreSchema::try_from(arrow_schema)
.context(error::ConvertStoreSchemaSnafu { file: &file_path })?,
);
let adapter = ReadAdapter::new(store_schema.clone(), self.projected_schema.clone())?;
@@ -290,14 +303,13 @@ impl<'a> ParquetReader<'a> {
builder = builder.with_row_filter(row_filter);
}
let mut stream = builder.build().context(ReadParquetSnafu {
file: self.file_path,
})?;
let mut stream = builder
.build()
.context(ReadParquetSnafu { file: &file_path })?;
let file_name = self.file_path.to_string();
let chunk_stream = try_stream!({
while let Some(res) = stream.next().await {
yield res.context(ReadParquetSnafu { file: &file_name })?
yield res.context(ReadParquetSnafu { file: &file_path })?
}
});
@@ -539,10 +551,12 @@ mod tests {
use store_api::storage::OpType;
use super::*;
use crate::file_purger::noop::new_noop_file_purger;
use crate::memtable::{
tests as memtable_tests, DefaultMemtableBuilder, IterContext, MemtableBuilder,
};
use crate::schema::ProjectedSchema;
use crate::sst::{FileId, FileMeta};
#[tokio::test]
async fn test_parquet_writer() {
@@ -673,9 +687,10 @@ mod tests {
let path = dir.path().to_str().unwrap();
let backend = Fs::default().root(path).build().unwrap();
let object_store = ObjectStore::new(backend).finish();
let sst_file_name = "test-read-large.parquet";
let sst_file_handle = new_file_handle(FileId::random());
let sst_file_name = sst_file_handle.file_name();
let iter = memtable.iter(&IterContext::default()).unwrap();
let writer = ParquetWriter::new(sst_file_name, Source::Iter(iter), object_store.clone());
let writer = ParquetWriter::new(&sst_file_name, Source::Iter(iter), object_store.clone());
let SstInfo {
time_range,
@@ -683,6 +698,7 @@ mod tests {
} = writer
.write_sst(&sst::WriteOptions::default())
.await
.unwrap()
.unwrap();
assert_eq!(
@@ -703,7 +719,7 @@ mod tests {
let projected_schema = Arc::new(ProjectedSchema::new(schema, Some(vec![1])).unwrap());
let reader = ParquetReader::new(
"test-read-large.parquet",
sst_file_handle,
operator,
projected_schema,
Predicate::empty(),
@@ -718,6 +734,25 @@ mod tests {
assert_eq!(rows_total, rows_fetched);
}
fn new_file_handle(file_id: FileId) -> FileHandle {
let file_purger = new_noop_file_purger();
let layer = Arc::new(crate::test_util::access_layer_util::MockAccessLayer {});
FileHandle::new(
FileMeta {
region_id: 0,
file_id,
time_range: Some((
Timestamp::new_millisecond(0),
Timestamp::new_millisecond(1000),
)),
level: 0,
file_size: 0,
},
layer,
file_purger,
)
}
#[tokio::test]
async fn test_parquet_reader() {
common_telemetry::init_default_ut_logging();
@@ -750,9 +785,10 @@ mod tests {
let path = dir.path().to_str().unwrap();
let backend = Fs::default().root(path).build().unwrap();
let object_store = ObjectStore::new(backend).finish();
let sst_file_name = "test-read.parquet";
let file_handle = new_file_handle(FileId::random());
let sst_file_name = file_handle.file_name();
let iter = memtable.iter(&IterContext::default()).unwrap();
let writer = ParquetWriter::new(sst_file_name, Source::Iter(iter), object_store.clone());
let writer = ParquetWriter::new(&sst_file_name, Source::Iter(iter), object_store.clone());
let SstInfo {
time_range,
@@ -760,6 +796,7 @@ mod tests {
} = writer
.write_sst(&sst::WriteOptions::default())
.await
.unwrap()
.unwrap();
assert_eq!(
@@ -780,7 +817,7 @@ mod tests {
let projected_schema = Arc::new(ProjectedSchema::new(schema, Some(vec![1])).unwrap());
let reader = ParquetReader::new(
"test-read.parquet",
file_handle,
operator,
projected_schema,
Predicate::empty(),
@@ -801,17 +838,18 @@ mod tests {
}
async fn check_range_read(
file_name: &str,
file_handle: FileHandle,
object_store: ObjectStore,
schema: ProjectedSchemaRef,
range: TimestampRange,
expect: Vec<i64>,
) {
let reader = ParquetReader::new(file_name, object_store, schema, Predicate::empty(), range);
let reader =
ParquetReader::new(file_handle, object_store, schema, Predicate::empty(), range);
let mut stream = reader.chunk_stream().await.unwrap();
let result = stream.next_batch().await;
let Some(batch) = result.unwrap() else {
let Some(batch) = result.unwrap() else {
// if batch does not contain any row
assert!(expect.is_empty());
return;
@@ -867,9 +905,10 @@ mod tests {
let path = dir.path().to_str().unwrap();
let backend = Fs::default().root(path).build().unwrap();
let object_store = ObjectStore::new(backend).finish();
let sst_file_name = "test-read.parquet";
let sst_file_handle = new_file_handle(FileId::random());
let sst_file_name = sst_file_handle.file_name();
let iter = memtable.iter(&IterContext::default()).unwrap();
let writer = ParquetWriter::new(sst_file_name, Source::Iter(iter), object_store.clone());
let writer = ParquetWriter::new(&sst_file_name, Source::Iter(iter), object_store.clone());
let SstInfo {
time_range,
@@ -877,6 +916,7 @@ mod tests {
} = writer
.write_sst(&sst::WriteOptions::default())
.await
.unwrap()
.unwrap();
assert_eq!(
@@ -892,7 +932,7 @@ mod tests {
Arc::new(ProjectedSchema::new(schema, Some(vec![1, 0, 3, 2])).unwrap());
check_range_read(
sst_file_name,
sst_file_handle.clone(),
object_store.clone(),
projected_schema.clone(),
TimestampRange::with_unit(1000, 2003, TimeUnit::Millisecond).unwrap(),
@@ -901,7 +941,7 @@ mod tests {
.await;
check_range_read(
sst_file_name,
sst_file_handle.clone(),
object_store.clone(),
projected_schema.clone(),
TimestampRange::with_unit(2002, 3001, TimeUnit::Millisecond).unwrap(),
@@ -911,7 +951,7 @@ mod tests {
// read a range without any rows.
check_range_read(
sst_file_name,
sst_file_handle.clone(),
object_store.clone(),
projected_schema.clone(),
TimestampRange::with_unit(3002, 3003, TimeUnit::Millisecond).unwrap(),
@@ -921,7 +961,7 @@ mod tests {
//
check_range_read(
sst_file_name,
sst_file_handle.clone(),
object_store.clone(),
projected_schema.clone(),
TimestampRange::with_unit(1000, 3000, TimeUnit::Millisecond).unwrap(),
@@ -931,7 +971,7 @@ mod tests {
// read full range
check_range_read(
sst_file_name,
sst_file_handle,
object_store,
projected_schema,
TimestampRange::min_to_max(),
@@ -950,6 +990,28 @@ mod tests {
)
}
#[tokio::test]
async fn test_write_empty_file() {
common_telemetry::init_default_ut_logging();
let schema = memtable_tests::schema_for_test();
let memtable = DefaultMemtableBuilder::default().build(schema.clone());
let dir = create_temp_dir("read-parquet-by-range");
let path = dir.path().to_str().unwrap();
let backend = Fs::default().root(path).build().unwrap();
let object_store = ObjectStore::new(backend).finish();
let sst_file_name = "test-read.parquet";
let iter = memtable.iter(&IterContext::default()).unwrap();
let writer = ParquetWriter::new(sst_file_name, Source::Iter(iter), object_store.clone());
let sst_info_opt = writer
.write_sst(&sst::WriteOptions::default())
.await
.unwrap();
assert!(sst_info_opt.is_none());
}
#[test]
fn test_time_unit_lossy() {
// converting a range with unit second to millisecond will not cause rounding error

View File

@@ -13,25 +13,29 @@
// limitations under the License.
use crate::read::BoxedBatchReader;
use crate::sst::{AccessLayer, FileId, ReadOptions, Source, SstInfo, WriteOptions};
use crate::sst::{AccessLayer, FileHandle, FileId, ReadOptions, Source, SstInfo, WriteOptions};
#[derive(Debug)]
pub struct MockAccessLayer;
#[async_trait::async_trait]
impl AccessLayer for MockAccessLayer {
fn sst_file_path(&self, file_name: &str) -> String {
file_name.to_string()
}
async fn write_sst(
&self,
_file_id: FileId,
_source: Source,
_opts: &WriteOptions,
) -> crate::error::Result<SstInfo> {
) -> crate::error::Result<Option<SstInfo>> {
unimplemented!()
}
async fn read_sst(
&self,
_file_id: FileId,
_file_handle: FileHandle,
_opts: &ReadOptions,
) -> crate::error::Result<BoxedBatchReader> {
unimplemented!()

View File

@@ -13,6 +13,7 @@
// limitations under the License.
//! Table and TableEngine requests
use std::collections::HashMap;
use std::str::FromStr;
use std::time::Duration;
@@ -27,16 +28,6 @@ use crate::error;
use crate::error::ParseTableOptionSnafu;
use crate::metadata::TableId;
/// Insert request
#[derive(Debug)]
pub struct InsertRequest {
pub catalog_name: String,
pub schema_name: String,
pub table_name: String,
pub columns_values: HashMap<String, VectorRef>,
pub region_number: RegionNumber,
}
#[derive(Debug, Clone)]
pub struct CreateDatabaseRequest {
pub db_name: String,
@@ -181,6 +172,15 @@ pub struct DropTableRequest {
pub table_name: String,
}
#[derive(Debug)]
pub struct InsertRequest {
pub catalog_name: String,
pub schema_name: String,
pub table_name: String,
pub columns_values: HashMap<String, VectorRef>,
pub region_number: RegionNumber,
}
/// Delete (by primary key) request
#[derive(Debug)]
pub struct DeleteRequest {

View File

@@ -3,7 +3,7 @@
## Sqlness manual
### Case file
Sqlness has three types of file
Sqlness has two types of file
- `.sql`: test input, SQL only
- `.result`: expected test output, SQL and its results

View File

@@ -0,0 +1,45 @@
create table demo1(host string, cpu double, memory double, ts timestamp time index);
Affected Rows: 0
create table demo2(host string, cpu double, memory double, ts timestamp time index);
Affected Rows: 0
insert into demo1(host, cpu, memory, ts) values ('host1', 66.6, 1024, 1655276557000), ('host2', 88.8, 333.3, 1655276558000);
Affected Rows: 2
insert into demo2(host) select * from demo1;
Error: 3000(PlanQuery), Error during planning: Column count doesn't match insert query!
insert into demo2 select cpu,memory from demo1;
Error: 3000(PlanQuery), Error during planning: Column count doesn't match insert query!
insert into demo2(ts) select memory from demo1;
Error: 3000(PlanQuery), Error during planning: Cannot automatically convert Float64 to Timestamp(Millisecond, None)
insert into demo2 select * from demo1;
Affected Rows: 2
select * from demo2 order by ts;
+-------+------+--------+---------------------+
| host | cpu | memory | ts |
+-------+------+--------+---------------------+
| host1 | 66.6 | 1024.0 | 2022-06-15T07:02:37 |
| host2 | 88.8 | 333.3 | 2022-06-15T07:02:38 |
+-------+------+--------+---------------------+
drop table demo1;
Affected Rows: 1
drop table demo2;
Affected Rows: 1

View File

@@ -0,0 +1,19 @@
create table demo1(host string, cpu double, memory double, ts timestamp time index);
create table demo2(host string, cpu double, memory double, ts timestamp time index);
insert into demo1(host, cpu, memory, ts) values ('host1', 66.6, 1024, 1655276557000), ('host2', 88.8, 333.3, 1655276558000);
insert into demo2(host) select * from demo1;
insert into demo2 select cpu,memory from demo1;
insert into demo2(ts) select memory from demo1;
insert into demo2 select * from demo1;
select * from demo2 order by ts;
drop table demo1;
drop table demo2;

View File

@@ -34,3 +34,11 @@ select * where "a" = "A";
Error: 3000(PlanQuery), No field named 'a'.
select TO_UNIXTIME('2023-03-01T06:35:02Z');
+-------------------------------------------+
| to_unixtime(Utf8("2023-03-01T06:35:02Z")) |
+-------------------------------------------+
| 1677652502 |
+-------------------------------------------+

View File

@@ -9,3 +9,5 @@ select "a";
select "A";
select * where "a" = "A";
select TO_UNIXTIME('2023-03-01T06:35:02Z');

View File

@@ -8,19 +8,19 @@ Affected Rows: 3
delete from monitor where cpu = 66.6 and ts = 1655276557000;
Error: 1004(InvalidArguments), Missing column host in write batch
Affected Rows: 1
delete from monitor where host = 'host1' or ts = 1655276557000;
Error: 1004(InvalidArguments), Not support SQL, error: Not support sql expr:host = 'host1' OR ts = 1655276557000,correct format is tagkey1 = tagvalue1 and ts = value
Affected Rows: 0
delete from monitor where host = 'host1' or ts != 1655276557000;
Error: 1004(InvalidArguments), Not support SQL, error: Not support sql expr:host = 'host1' OR ts <> 1655276557000,correct format is tagkey1 = tagvalue1 and ts = value
Affected Rows: 2
delete from monitor where ts != 1655276557000;
Error: 1004(InvalidArguments), Not support SQL, error: Not support sql expr:ts <> 1655276557000,correct format is tagkey1 = tagvalue1 and ts = value
Affected Rows: 0
drop table monitor;