mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2025-12-26 08:00:01 +00:00
Compare commits
14 Commits
v0.1.2
...
v0.2.0-nig
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
b4fc8c5b78 | ||
|
|
6f81717866 | ||
|
|
77f9383daf | ||
|
|
c788b7fc26 | ||
|
|
0f160a73be | ||
|
|
92963b9614 | ||
|
|
f1139fba59 | ||
|
|
4e552245b1 | ||
|
|
3126bbc1c7 | ||
|
|
b77b561bc8 | ||
|
|
501faad8ab | ||
|
|
5397a9bbe6 | ||
|
|
f351ee7042 | ||
|
|
e0493e0b8f |
1
Cargo.lock
generated
1
Cargo.lock
generated
@@ -2294,6 +2294,7 @@ dependencies = [
|
||||
"common-catalog",
|
||||
"common-datasource",
|
||||
"common-error",
|
||||
"common-function",
|
||||
"common-grpc",
|
||||
"common-grpc-expr",
|
||||
"common-procedure",
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -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()));
|
||||
}
|
||||
}
|
||||
|
||||
148
src/common/function/src/scalars/timestamp/to_unixtime.rs
Normal file
148
src/common/function/src/scalars/timestamp/to_unixtime.rs
Normal 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!(),
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -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 {
|
||||
|
||||
@@ -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" }
|
||||
|
||||
@@ -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 { .. }
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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")]
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -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!(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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(),
|
||||
}
|
||||
}
|
||||
@@ -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 })
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -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,
|
||||
}
|
||||
|
||||
@@ -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 {
|
||||
|
||||
@@ -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(©_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();
|
||||
|
||||
@@ -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)?,
|
||||
|
||||
@@ -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();
|
||||
|
||||
@@ -28,7 +28,6 @@ pub mod postgres;
|
||||
pub mod prom;
|
||||
pub mod prometheus;
|
||||
mod server;
|
||||
mod sql;
|
||||
mod table;
|
||||
#[cfg(test)]
|
||||
mod tests;
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
@@ -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,
|
||||
};
|
||||
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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 {
|
||||
|
||||
@@ -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);
|
||||
|
||||
151
src/meta-srv/src/handler/failure_handler.rs
Normal file
151
src/meta-srv/src/handler/failure_handler.rs
Normal 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);
|
||||
}
|
||||
}
|
||||
309
src/meta-srv/src/handler/failure_handler/runner.rs
Normal file
309
src/meta-srv/src/handler/failure_handler/runner.rs
Normal 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(|®ion_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();
|
||||
}
|
||||
}
|
||||
@@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -65,6 +65,7 @@ mod tests {
|
||||
catalog: None,
|
||||
schema: None,
|
||||
table: None,
|
||||
is_infancy: false,
|
||||
};
|
||||
|
||||
let req = HeartbeatRequest {
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -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,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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;
|
||||
|
||||
|
||||
16
src/promql/src/functions/changes.rs
Normal file
16
src/promql/src/functions/changes.rs
Normal 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).
|
||||
16
src/promql/src/functions/deriv.rs
Normal file
16
src/promql/src/functions/deriv.rs
Normal 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).
|
||||
16
src/promql/src/functions/resets.rs
Normal file
16
src/promql/src/functions/resets.rs
Normal 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).
|
||||
@@ -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(
|
||||
|
||||
@@ -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) => {
|
||||
|
||||
@@ -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,
|
||||
}
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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);
|
||||
|
||||
|
||||
@@ -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()
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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())
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -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(),
|
||||
)),
|
||||
})?
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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! {
|
||||
|
||||
@@ -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));
|
||||
|
||||
@@ -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__")
|
||||
|
||||
@@ -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]);
|
||||
|
||||
@@ -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())
|
||||
}
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -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))
|
||||
|
||||
@@ -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 })
|
||||
}
|
||||
|
||||
@@ -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 {
|
||||
|
||||
@@ -167,7 +167,7 @@ impl ScriptsTable {
|
||||
|
||||
let stream = match self
|
||||
.query_engine
|
||||
.execute(&plan)
|
||||
.execute(plan, QueryContext::arc())
|
||||
.await
|
||||
.context(FindScriptSnafu { name })?
|
||||
{
|
||||
|
||||
@@ -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, ()))
|
||||
}
|
||||
|
||||
@@ -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)]
|
||||
}
|
||||
|
||||
|
||||
@@ -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(),
|
||||
|
||||
@@ -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,
|
||||
}
|
||||
|
||||
@@ -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:?}"),
|
||||
|
||||
@@ -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);
|
||||
}
|
||||
|
||||
@@ -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]
|
||||
|
||||
@@ -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,
|
||||
},
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -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!(
|
||||
|
||||
@@ -145,6 +145,7 @@ mod tests {
|
||||
let sst_info = layer
|
||||
.write_sst(sst_file_id, Source::Iter(iter), &WriteOptions {})
|
||||
.await
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
|
||||
(
|
||||
|
||||
@@ -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);
|
||||
|
||||
@@ -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);
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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!()
|
||||
|
||||
@@ -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 {
|
||||
|
||||
@@ -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
|
||||
|
||||
|
||||
45
tests/cases/standalone/common/insert/insert_select.result
Normal file
45
tests/cases/standalone/common/insert/insert_select.result
Normal 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
|
||||
|
||||
19
tests/cases/standalone/common/insert/insert_select.sql
Normal file
19
tests/cases/standalone/common/insert/insert_select.sql
Normal 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;
|
||||
@@ -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 |
|
||||
+-------------------------------------------+
|
||||
|
||||
|
||||
@@ -9,3 +9,5 @@ select "a";
|
||||
select "A";
|
||||
|
||||
select * where "a" = "A";
|
||||
|
||||
select TO_UNIXTIME('2023-03-01T06:35:02Z');
|
||||
|
||||
@@ -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;
|
||||
|
||||
|
||||
Reference in New Issue
Block a user