From 8dd4371d5176934e04e751330cf443e4b9189041 Mon Sep 17 00:00:00 2001 From: flaneur Date: Fri, 20 Oct 2023 07:58:15 +0800 Subject: [PATCH 01/39] fix(metrics): Recover truncate metrics (#13296) * add wrapped registry * fix cargo check * add resettable counter * seems works now * rename counter.rs to count.rs * rename resettable to counter.rs * add comments * fix typo * fix lincense * add parkinglot; port the histogram * reset histogram * tune comment * fix lint * taplo fmt * add one sqlogictest * try fix sqlogictest * fix sqlogictest * use session_connect_numbers_total as test * try another sqlogic test * fix * try again --- Cargo.lock | 1 + src/common/metrics/Cargo.toml | 1 + src/common/metrics/src/count.rs | 139 +++++++++++++++++ src/common/metrics/src/counter.rs | 144 +++++------------- src/common/metrics/src/histogram.rs | 114 ++++++++++++++ src/common/metrics/src/lib.rs | 5 +- src/common/metrics/src/registry.rs | 91 ++++++++++- src/common/metrics/tests/it/main.rs | 2 +- src/common/storage/src/operator.rs | 2 +- src/meta/service/src/api/grpc/grpc_service.rs | 2 +- .../service/src/meta_service/meta_leader.rs | 2 +- src/meta/service/src/metrics/meta_metrics.rs | 6 +- src/meta/service/src/raft_client.rs | 8 +- .../base/01_system/01_0013_system_metrics | 56 +++++++ 14 files changed, 450 insertions(+), 123 deletions(-) create mode 100644 src/common/metrics/src/count.rs create mode 100644 tests/sqllogictests/suites/base/01_system/01_0013_system_metrics diff --git a/Cargo.lock b/Cargo.lock index e0ac3289f4d7..90728721d897 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2409,6 +2409,7 @@ dependencies = [ "lazy_static", "metrics", "metrics-exporter-prometheus", + "parking_lot 0.12.1", "procfs 0.15.1", "prometheus-client", "prometheus-parse", diff --git a/src/common/metrics/Cargo.toml b/src/common/metrics/Cargo.toml index d1094050673a..39cea1c2d157 100644 --- a/src/common/metrics/Cargo.toml +++ b/src/common/metrics/Cargo.toml @@ -21,6 +21,7 @@ common-exception = { path = "../exception" } lazy_static = { workspace = true } metrics = "0.20.1" metrics-exporter-prometheus = { version = "0.11.0", default-features = false } +parking_lot = "0.12.1" prometheus-client = { workspace = true } prometheus-parse = "0.2.3" serde = { workspace = true } diff --git a/src/common/metrics/src/count.rs b/src/common/metrics/src/count.rs new file mode 100644 index 000000000000..70a6077298f6 --- /dev/null +++ b/src/common/metrics/src/count.rs @@ -0,0 +1,139 @@ +// Copyright 2021 Datafuse Labs +// +// 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. + +//! This mod provides mechanism to track the count of active instances of some type `T`. +//! The count is maintained by a `Count` implementation and will be increased or decreased when a wrapper of `T` `WithCounter` is created or dropped. +//! +//! Example: +//! +//! ```ignore +//! struct Connection{} +//! impl Connection { +//! fn ping() {} +//! } +//! +//! struct MyCounter{ identifier: String, } +//! impl Count for MyCounter {/*...*/} +//! +//! { +//! let conn = WithCounter::new(Connection{}, MyCounter{}); // increase count with `MyCounter` +//! conn.ping(); +//! } // decrease count with `MyCounter` +//! ``` + +use std::ops::Deref; +use std::ops::DerefMut; + +/// Defines how to report counter metrics. +pub trait Count { + fn incr_count(&mut self, n: i64); + + /// Create a guard instance that increases the counter when created, and decreases the counter when dropped. + fn guard() -> WithCount + where Self: Default + Sized { + WithCount::new((), Self::default()) + } +} + +/// Binds a counter to a `T`. +/// +/// It counts the number of instances of `T` with the provided counter `Count`. +#[derive(Debug)] +pub struct WithCount +where C: Count +{ + counter: C, + inner: T, +} + +impl WithCount +where C: Count +{ + pub fn new(t: T, counter: C) -> Self { + let mut s = Self { counter, inner: t }; + s.counter.incr_count(1); + s + } + + pub fn counter(&self) -> &C { + &self.counter + } +} + +/// When being dropped, decreases the count. +impl Drop for WithCount +where C: Count +{ + fn drop(&mut self) { + self.counter.incr_count(-1); + } +} + +/// Let an app use `WithCount` the same as using `T`. +impl Deref for WithCount +where C: Count +{ + type Target = T; + + fn deref(&self) -> &Self::Target { + &self.inner + } +} + +/// Let an app use `WithCount` the same as using `T`. +impl DerefMut for WithCount +where C: Count +{ + fn deref_mut(&mut self) -> &mut Self::Target { + &mut self.inner + } +} + +#[cfg(test)] +mod tests { + use std::sync::atomic::AtomicI64; + use std::sync::atomic::Ordering; + use std::sync::Arc; + + use crate::count::Count; + use crate::count::WithCount; + + struct Foo {} + struct Counter { + n: Arc, + } + impl Count for Counter { + fn incr_count(&mut self, n: i64) { + self.n.fetch_add(n, Ordering::Relaxed); + } + } + + #[test] + fn test_with_count() -> anyhow::Result<()> { + let count = Arc::new(AtomicI64::new(0)); + assert_eq!(0, count.load(Ordering::Relaxed)); + + { + let _a = WithCount::new(Foo {}, Counter { n: count.clone() }); + assert_eq!(1, count.load(Ordering::Relaxed)); + { + let _b = WithCount::new(Foo {}, Counter { n: count.clone() }); + assert_eq!(2, count.load(Ordering::Relaxed)); + } + assert_eq!(1, count.load(Ordering::Relaxed)); + } + assert_eq!(0, count.load(Ordering::Relaxed)); + Ok(()) + } +} diff --git a/src/common/metrics/src/counter.rs b/src/common/metrics/src/counter.rs index 17a588fbcaa2..28351262f8f1 100644 --- a/src/common/metrics/src/counter.rs +++ b/src/common/metrics/src/counter.rs @@ -12,128 +12,68 @@ // See the License for the specific language governing permissions and // limitations under the License. -//! This mod provides mechanism to track the count of active instances of some type `T`. -//! The count is maintained by a `Count` implementation and will be increased or decreased when a wrapper of `T` `WithCounter` is created or dropped. -//! -//! Example: -//! -//! ```ignore -//! struct Connection{} -//! impl Connection { -//! fn ping() {} -//! } -//! -//! struct MyCounter{ identifier: String, } -//! impl Count for MyCounter {/*...*/} -//! -//! { -//! let conn = WithCounter::new(Connection{}, MyCounter{}); // increase count with `MyCounter` -//! conn.ping(); -//! } // decrease count with `MyCounter` -//! ``` +use std::sync::atomic::AtomicU64; +use std::sync::atomic::Ordering; +use std::sync::Arc; -use std::ops::Deref; -use std::ops::DerefMut; +use prometheus_client::encoding::EncodeMetric; +use prometheus_client::encoding::MetricEncoder; +use prometheus_client::metrics::MetricType; +use prometheus_client::metrics::TypedMetric; -/// Defines how to report counter metrics. -pub trait Count { - fn incr_count(&mut self, n: i64); - - /// Create a guard instance that increases the counter when created, and decreases the counter when dropped. - fn guard() -> WithCount - where Self: Default + Sized { - WithCount::new((), Self::default()) - } -} - -/// Binds a counter to a `T`. -/// -/// It counts the number of instances of `T` with the provided counter `Count`. #[derive(Debug)] -pub struct WithCount -where C: Count -{ - counter: C, - inner: T, +pub struct Counter { + value: Arc, } -impl WithCount -where C: Count -{ - pub fn new(t: T, counter: C) -> Self { - let mut s = Self { counter, inner: t }; - s.counter.incr_count(1); - s +impl Clone for Counter { + fn clone(&self) -> Self { + Self { + value: self.value.clone(), + } } +} - pub fn counter(&self) -> &C { - &self.counter +impl Default for Counter { + fn default() -> Self { + Counter { + value: Arc::new(AtomicU64::new(0)), + } } } -/// When being dropped, decreases the count. -impl Drop for WithCount -where C: Count -{ - fn drop(&mut self) { - self.counter.incr_count(-1); +impl Counter { + /// Increase the [`Counter`] by 1, returning the previous value. + pub fn inc(&self) -> u64 { + self.value.fetch_add(1, Ordering::Relaxed) } -} -/// Let an app use `WithCount` the same as using `T`. -impl Deref for WithCount -where C: Count -{ - type Target = T; + /// Increase the [`Counter`] by `v`, returning the previous value. + pub fn inc_by(&self, v: u64) -> u64 { + self.value.fetch_add(v, Ordering::Relaxed) + } - fn deref(&self) -> &Self::Target { - &self.inner + /// Get the current value of the [`Counter`]. + pub fn get(&self) -> u64 { + self.value.load(Ordering::Relaxed) } -} -/// Let an app use `WithCount` the same as using `T`. -impl DerefMut for WithCount -where C: Count -{ - fn deref_mut(&mut self) -> &mut Self::Target { - &mut self.inner + /// Reset the [`Counter`] to 0. + pub fn reset(&self) { + self.value.store(0, Ordering::Release) } } -#[cfg(test)] -mod tests { - use std::sync::atomic::AtomicI64; - use std::sync::atomic::Ordering; - use std::sync::Arc; - - use crate::counter::Count; - use crate::counter::WithCount; +impl TypedMetric for Counter { + const TYPE: MetricType = MetricType::Counter; +} - struct Foo {} - struct Counter { - n: Arc, - } - impl Count for Counter { - fn incr_count(&mut self, n: i64) { - self.n.fetch_add(n, Ordering::Relaxed); - } +impl EncodeMetric for Counter { + fn encode(&self, mut encoder: MetricEncoder) -> Result<(), std::fmt::Error> { + encoder.encode_counter::<(), _, u64>(&self.get(), None) } - #[test] - fn test_with_count() -> anyhow::Result<()> { - let count = Arc::new(AtomicI64::new(0)); - assert_eq!(0, count.load(Ordering::Relaxed)); - - { - let _a = WithCount::new(Foo {}, Counter { n: count.clone() }); - assert_eq!(1, count.load(Ordering::Relaxed)); - { - let _b = WithCount::new(Foo {}, Counter { n: count.clone() }); - assert_eq!(2, count.load(Ordering::Relaxed)); - } - assert_eq!(1, count.load(Ordering::Relaxed)); - } - assert_eq!(0, count.load(Ordering::Relaxed)); - Ok(()) + fn metric_type(&self) -> MetricType { + Self::TYPE } } diff --git a/src/common/metrics/src/histogram.rs b/src/common/metrics/src/histogram.rs index 0ede7df0cf73..73d6f5deeebf 100644 --- a/src/common/metrics/src/histogram.rs +++ b/src/common/metrics/src/histogram.rs @@ -12,6 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::iter::once; +use std::sync::Arc; + +use parking_lot::MappedRwLockReadGuard; +use parking_lot::RwLock; +use parking_lot::RwLockReadGuard; +use prometheus_client::encoding::EncodeMetric; +use prometheus_client::encoding::MetricEncoder; +use prometheus_client::metrics::MetricType; +use prometheus_client::metrics::TypedMetric; + pub static BUCKET_SECONDS: [f64; 15] = [ 0.02, 0.05, 0.1, 0.25, 0.5, 1.0, 2.5, 5.0, 10.0, 20.0, 30.0, 60.0, 300.0, 600.0, 1800.0, ]; @@ -20,3 +31,106 @@ pub static BUCKET_MILLISECONDS: [f64; 15] = [ 10.0, 50.0, 100.0, 250.0, 500.0, 1000.0, 2500.0, 5000.0, 10000.0, 20000.0, 30000.0, 60000.0, 300000.0, 600000.0, 1800000.0, ]; + +/// Histogram is a port of prometheus-client's Histogram. The only difference is that +/// we can reset the histogram. +#[derive(Debug)] +pub struct Histogram { + inner: Arc>, +} + +impl Clone for Histogram { + fn clone(&self) -> Self { + Histogram { + inner: self.inner.clone(), + } + } +} + +#[derive(Debug)] +pub(crate) struct Inner { + sum: f64, + count: u64, + buckets: Vec<(f64, u64)>, +} + +impl Histogram { + /// Create a new [`Histogram`]. + pub fn new(buckets: impl Iterator) -> Self { + Self { + inner: Arc::new(RwLock::new(Inner { + sum: Default::default(), + count: Default::default(), + buckets: buckets + .into_iter() + .chain(once(f64::MAX)) + .map(|upper_bound| (upper_bound, 0)) + .collect(), + })), + } + } + + /// Observe the given value. + pub fn observe(&self, v: f64) { + self.observe_and_bucket(v); + } + + /// Observes the given value, returning the index of the first bucket the + /// value is added to. + /// + /// Needed in + /// [`HistogramWithExemplars`](crate::metrics::exemplar::HistogramWithExemplars). + pub(crate) fn observe_and_bucket(&self, v: f64) -> Option { + let mut inner = self.inner.write(); + inner.sum += v; + inner.count += 1; + + let first_bucket = inner + .buckets + .iter_mut() + .enumerate() + .find(|(_i, (upper_bound, _value))| upper_bound >= &v); + + match first_bucket { + Some((i, (_upper_bound, value))) => { + *value += 1; + Some(i) + } + None => None, + } + } + + pub(crate) fn get(&self) -> (f64, u64, MappedRwLockReadGuard>) { + let inner = self.inner.read(); + let sum = inner.sum; + let count = inner.count; + let buckets = RwLockReadGuard::map(inner, |inner| &inner.buckets); + (sum, count, buckets) + } + + pub(crate) fn reset(&self) { + let mut inner = self.inner.write(); + inner.sum = 0.0; + inner.count = 0; + inner.buckets = inner + .buckets + .iter() + .map(|(upper_bound, _value)| (*upper_bound, 0)) + .collect(); + } +} + +impl TypedMetric for Histogram { + const TYPE: MetricType = MetricType::Histogram; +} + +impl EncodeMetric for Histogram { + fn encode(&self, mut encoder: MetricEncoder) -> Result<(), std::fmt::Error> { + let (sum, count, buckets) = self.get(); + encoder.encode_histogram::<()>(sum, count, &buckets, None) + } + + fn metric_type(&self) -> MetricType { + Self::TYPE + } +} diff --git a/src/common/metrics/src/lib.rs b/src/common/metrics/src/lib.rs index 066ee962b0b3..38d4a1d26ede 100644 --- a/src/common/metrics/src/lib.rs +++ b/src/common/metrics/src/lib.rs @@ -14,6 +14,7 @@ #![allow(clippy::uninlined_format_args)] +pub mod count; pub mod counter; mod dump; pub mod histogram; @@ -21,16 +22,16 @@ pub mod registry; pub type VecLabels = Vec<(&'static str, String)>; +pub use counter::Counter; pub use dump::dump_metric_samples; pub use dump::HistogramCount; pub use dump::MetricSample; pub use dump::MetricValue; pub use dump::SummaryCount; +pub use histogram::Histogram; pub use metrics_exporter_prometheus::PrometheusHandle; -pub use prometheus_client::metrics::counter::Counter; pub use prometheus_client::metrics::family::Family; pub use prometheus_client::metrics::gauge::Gauge; -pub use prometheus_client::metrics::histogram::Histogram; pub use registry::load_global_prometheus_registry; pub use registry::register_counter; pub use registry::register_counter_family; diff --git a/src/common/metrics/src/registry.rs b/src/common/metrics/src/registry.rs index a7de87108e80..16069a0e9373 100644 --- a/src/common/metrics/src/registry.rs +++ b/src/common/metrics/src/registry.rs @@ -12,34 +12,109 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::ops::Deref; use std::sync::Mutex; use std::sync::MutexGuard; use lazy_static::lazy_static; use prometheus_client::encoding::text::encode as prometheus_encode; use prometheus_client::encoding::EncodeLabelSet; -use prometheus_client::metrics::counter::Counter; use prometheus_client::metrics::family::Family; +use prometheus_client::metrics::family::MetricConstructor; use prometheus_client::metrics::gauge::Gauge; -use prometheus_client::metrics::histogram::Histogram; +use prometheus_client::registry::Metric; use prometheus_client::registry::Registry; +use crate::counter::Counter; +use crate::histogram::Histogram; use crate::histogram::BUCKET_MILLISECONDS; use crate::histogram::BUCKET_SECONDS; lazy_static! { - pub static ref REGISTRY: Mutex = Mutex::new(Registry::with_prefix("databend")); + pub static ref REGISTRY: Mutex = + Mutex::new(WrappedRegistry::with_prefix("databend")); } -pub fn load_global_prometheus_registry() -> MutexGuard<'static, Registry> { +pub fn load_global_prometheus_registry() -> MutexGuard<'static, WrappedRegistry> { REGISTRY.lock().unwrap() } pub fn reset_global_prometheus_registry() { - // TODO(liyz): do nothing yet. This function would be trivial once prometheus_client - // supports iterating metrics. However it's not supported yet. I've raised an issue about - // this: https://github.com/prometheus/client_rust/issues/163 . If this feature request - // got denied, we can still wrap a customized Registry which record the metrics by itself. + let mut registry = load_global_prometheus_registry(); + registry.reset(); +} + +pub trait ResetMetric { + fn reset_metric(&self); +} + +impl ResetMetric for Counter { + fn reset_metric(&self) { + self.reset() + } +} + +impl ResetMetric for Histogram { + fn reset_metric(&self) { + self.reset() + } +} + +impl ResetMetric for Gauge { + fn reset_metric(&self) { + let v = self.get(); + self.inc_by(-v); + } +} + +impl> ResetMetric for Family { + fn reset_metric(&self) { + self.clear(); + } +} + +/// [`WrappedRegistry`] wraps [`Registry`] and provides an additional reset method, which is useful +/// on `TRUNCATE system.metrics` on diagnosing customer issues. +pub struct WrappedRegistry { + inner: Registry, + resetters: Vec>, +} + +impl WrappedRegistry { + pub fn with_prefix(prefix: &str) -> Self { + let inner = Registry::with_prefix(prefix); + Self { + inner, + resetters: vec![], + } + } + + pub fn register(&mut self, name: &str, help: &str, metric: impl Metric + ResetMetric + Clone) { + self.resetters.push(Box::new(metric.clone())); + self.inner.register(name, help, metric); + } + + pub fn reset(&mut self) { + for resetter in &self.resetters { + resetter.reset_metric(); + } + } + + pub fn inner_mut(&mut self) -> &mut Registry { + &mut self.inner + } + + pub fn inner(&self) -> &Registry { + &self.inner + } +} + +impl Deref for WrappedRegistry { + type Target = Registry; + + fn deref(&self) -> &Self::Target { + &self.inner + } } pub fn render_prometheus_metrics(registry: &Registry) -> String { diff --git a/src/common/metrics/tests/it/main.rs b/src/common/metrics/tests/it/main.rs index 60dab279f8e2..283d387d6982 100644 --- a/src/common/metrics/tests/it/main.rs +++ b/src/common/metrics/tests/it/main.rs @@ -31,7 +31,7 @@ async fn test_dump_metric_samples() -> common_exception::Result<()> { histogram1.observe(2.0); let registry = load_global_prometheus_registry(); - let samples = dump_metric_samples(®istry) + let samples = dump_metric_samples(registry.inner()) .unwrap() .into_iter() .map(|s| (s.name.clone(), s)) diff --git a/src/common/storage/src/operator.rs b/src/common/storage/src/operator.rs index e8252a671ab7..87dd764d9d9d 100644 --- a/src/common/storage/src/operator.rs +++ b/src/common/storage/src/operator.rs @@ -113,7 +113,7 @@ pub fn build_operator(builder: B) -> Result { .layer(MinitraceLayer) // Add PrometheusClientLayer .layer(PrometheusClientLayer::new( - &mut load_global_prometheus_registry(), + load_global_prometheus_registry().inner_mut(), )) .finish(); diff --git a/src/meta/service/src/api/grpc/grpc_service.rs b/src/meta/service/src/api/grpc/grpc_service.rs index 886a24287ba2..10fa9040c7b1 100644 --- a/src/meta/service/src/api/grpc/grpc_service.rs +++ b/src/meta/service/src/api/grpc/grpc_service.rs @@ -40,7 +40,7 @@ use common_meta_types::protobuf::WatchRequest; use common_meta_types::protobuf::WatchResponse; use common_meta_types::TxnReply; use common_meta_types::TxnRequest; -use common_metrics::counter::Count; +use common_metrics::count::Count; use common_tracing::func_name; use futures::stream::TryChunksError; use futures::StreamExt; diff --git a/src/meta/service/src/meta_service/meta_leader.rs b/src/meta/service/src/meta_service/meta_leader.rs index a64a085553d7..baef38d3d71e 100644 --- a/src/meta/service/src/meta_service/meta_leader.rs +++ b/src/meta/service/src/meta_service/meta_leader.rs @@ -33,7 +33,7 @@ use common_meta_types::Node; use common_meta_types::NodeId; use common_meta_types::RaftError; use common_meta_types::SeqV; -use common_metrics::counter::Count; +use common_metrics::count::Count; use futures::StreamExt; use log::as_debug; use log::debug; diff --git a/src/meta/service/src/metrics/meta_metrics.rs b/src/meta/service/src/metrics/meta_metrics.rs index 2955ba96ef56..95f23efdc078 100644 --- a/src/meta/service/src/metrics/meta_metrics.rs +++ b/src/meta/service/src/metrics/meta_metrics.rs @@ -26,7 +26,7 @@ use std::time::Instant; -use common_metrics::counter; +use common_metrics::count; use prometheus_client::encoding::text::encode as prometheus_encode; pub mod server_metrics { @@ -617,7 +617,7 @@ pub(crate) struct RequestInFlight { start: Option, } -impl counter::Count for RequestInFlight { +impl count::Count for RequestInFlight { fn incr_count(&mut self, n: i64) { network_metrics::incr_request_inflights(n); @@ -636,7 +636,7 @@ impl counter::Count for RequestInFlight { #[derive(Default)] pub(crate) struct ProposalPending; -impl counter::Count for ProposalPending { +impl count::Count for ProposalPending { fn incr_count(&mut self, n: i64) { server_metrics::incr_proposals_pending(n); } diff --git a/src/meta/service/src/raft_client.rs b/src/meta/service/src/raft_client.rs index bf17deb6bde0..d6daedbe0a30 100644 --- a/src/meta/service/src/raft_client.rs +++ b/src/meta/service/src/raft_client.rs @@ -16,7 +16,7 @@ use common_meta_types::protobuf::raft_service_client::RaftServiceClient; use common_meta_types::Endpoint; use common_meta_types::GrpcConfig; use common_meta_types::NodeId; -use common_metrics::counter; +use common_metrics::count; use log::debug; use tonic::transport::channel::Channel; @@ -30,14 +30,14 @@ pub struct PeerCounter { endpoint_str: String, } -impl counter::Count for PeerCounter { +impl count::Count for PeerCounter { fn incr_count(&mut self, n: i64) { raft_metrics::network::incr_active_peers(&self.target, &self.endpoint_str, n) } } /// RaftClient is a grpc client bound with a metrics reporter.. -pub type RaftClient = counter::WithCount>; +pub type RaftClient = count::WithCount>; /// Defines the API of the client to a raft node. pub trait RaftClientApi { @@ -57,7 +57,7 @@ impl RaftClientApi for RaftClient { let cli = RaftServiceClient::new(channel) .max_decoding_message_size(GrpcConfig::MAX_DECODING_SIZE) .max_encoding_message_size(GrpcConfig::MAX_ENCODING_SIZE); - counter::WithCount::new(cli, PeerCounter { + count::WithCount::new(cli, PeerCounter { target, endpoint, endpoint_str, diff --git a/tests/sqllogictests/suites/base/01_system/01_0013_system_metrics b/tests/sqllogictests/suites/base/01_system/01_0013_system_metrics new file mode 100644 index 000000000000..911fa0b6aeac --- /dev/null +++ b/tests/sqllogictests/suites/base/01_system/01_0013_system_metrics @@ -0,0 +1,56 @@ +onlyif mysql +statement ok +---- +SELECT 1; + +onlyif mysql +statement ok +---- +SELECT 1; + +onlyif mysql +statement ok +---- +SELECT 1; + +onlyif mysql +statement ok +---- +SELECT 1; + +onlyif mysql +statement ok +---- +SELECT 1; + +onlyif mysql +statement ok +---- +SELECT 1; + +onlyif mysql +statement ok +---- +SELECT 1; + +onlyif mysql +statement ok +---- +SELECT 1; + +onlyif mysql +query I +SELECT sum(to_int32(to_float32(value) > 5)) > 1 FROM system.metrics where metric = 'query_success_total' +---- +1 + +onlyif mysql +statement ok +---- +truncate table system.metrics; + +onlyif mysql +query I +SELECT sum(to_int32(to_float32(value) > 5)) FROM system.metrics where metric = 'query_success_total' +---- +0 From 39edeba2cc5e4ae6d0a2de20a5999bf441d2f27f Mon Sep 17 00:00:00 2001 From: Andy Lok Date: Fri, 20 Oct 2023 08:00:08 +0800 Subject: [PATCH 02/39] fix: change evaluation error code to BadArgument (#13028) * fix: change eval error code to BadArgument * fix tests * fix tests * fix tests * fix tests --------- Co-authored-by: baishen --- src/query/expression/src/evaluator.rs | 6 +-- src/query/expression/src/function.rs | 2 +- .../sql/src/planner/semantic/type_check.rs | 2 +- .../03_common/03_0018_insert_into_variant | 4 +- .../base/03_common/03_0023_insert_into_array | 4 +- .../base/03_common/03_0027_insert_default | 2 +- .../base/03_common/03_0037_insert_into_map | 4 +- .../11_data_type/11_0001_data_type_date_time | 16 +++---- .../11_data_type/11_0006_data_type_decimal | 16 +++---- tests/sqllogictests/suites/crdb/and_or | 6 +-- tests/sqllogictests/suites/crdb/overflow | 4 +- .../duckdb/common/test_cast_hugeint.test | 4 +- .../monetdb/test_null_matches_outer.test | 4 +- .../issues/rigger/complex_division.test | 2 +- .../issues/rigger/division_by_zero_error.test | 4 +- .../sql/aggregate/aggregates/test_avg.test | 4 +- .../sql/cast/test_exponent_in_cast.test | 14 +++--- .../suites/duckdb/sql/cast/test_try_cast.test | 2 +- .../02_function/02_0000_function_arithmetic | 6 +-- .../query/02_function/02_0002_function_cast | 42 ++++++++-------- .../query/02_function/02_0002_function_others | 2 +- .../query/02_function/02_0010_function_if | 2 +- .../02_function/02_0012_function_datetimes | 48 +++++++++---------- .../02_function/02_0012_function_datetimes_tz | 6 +-- .../02_0013_function_running_difference | 2 +- .../query/02_function/02_0014_function_maths | 12 ++--- .../02_0022_function_strings_unhex | 2 +- .../02_0024_function_strings_base_64 | 2 +- .../02_0034_function_strings_space | 2 +- .../02_function/02_0035_function_strings_pad | 12 ++--- ..._0048_function_semi_structureds_parse_json | 4 +- .../02_0051_function_semi_structureds_get | 8 ++-- .../02_0052_function_string_regexp_substr | 2 +- .../02_0054_function_string_regexp_replace | 6 +-- .../query/02_function/02_0065_function_json | 4 +- .../suites/query/functions/auto_parse.test | 2 +- ...0005_alter_table_modify_column_type.result | 4 +- .../20+_others/20_0013_pretty_error.result | 2 +- 38 files changed, 135 insertions(+), 135 deletions(-) diff --git a/src/query/expression/src/evaluator.rs b/src/query/expression/src/evaluator.rs index 60f4e27dc2fd..e969bea40dcd 100644 --- a/src/query/expression/src/evaluator.rs +++ b/src/query/expression/src/evaluator.rs @@ -369,7 +369,7 @@ impl<'a> Evaluator<'a> { .map(|validity| validity.unset_bits() < validity.len()) .unwrap_or(true); if has_valid { - Err(ErrorCode::Internal(format!( + Err(ErrorCode::BadArguments(format!( "unable to cast type `NULL` to type `{dest_type}`" )) .set_span(span)) @@ -581,7 +581,7 @@ impl<'a> Evaluator<'a> { } } - _ => Err(ErrorCode::Internal(format!( + _ => Err(ErrorCode::BadArguments(format!( "unable to cast type `{src_type}` to type `{dest_type}`" )) .set_span(span)), @@ -763,7 +763,7 @@ impl<'a> Evaluator<'a> { } } - _ => Err(ErrorCode::Internal(format!( + _ => Err(ErrorCode::BadArguments(format!( "unable to cast type `{src_type}` to type `{dest_type}`" )) .set_span(span)), diff --git a/src/query/expression/src/function.rs b/src/query/expression/src/function.rs index 5b614705c99f..d57c01574b32 100755 --- a/src/query/expression/src/function.rs +++ b/src/query/expression/src/function.rs @@ -590,7 +590,7 @@ impl<'a> EvalContext<'a> { ) }; - Err(ErrorCode::Internal(err_msg).set_span(span)) + Err(ErrorCode::BadArguments(err_msg).set_span(span)) } None => Ok(()), } diff --git a/src/query/sql/src/planner/semantic/type_check.rs b/src/query/sql/src/planner/semantic/type_check.rs index 4181782a3467..0859137c1355 100644 --- a/src/query/sql/src/planner/semantic/type_check.rs +++ b/src/query/sql/src/planner/semantic/type_check.rs @@ -3659,7 +3659,7 @@ pub fn resolve_type_name_inner(type_name: &TypeName) -> Result { TableDataType::Map(Box::new(inner_type)) } _ => { - return Err(ErrorCode::Internal(format!( + return Err(ErrorCode::BadArguments(format!( "Invalid Map key type \'{:?}\'", key_type ))); diff --git a/tests/sqllogictests/suites/base/03_common/03_0018_insert_into_variant b/tests/sqllogictests/suites/base/03_common/03_0018_insert_into_variant index b6e9702072c5..3eb3952193b3 100644 --- a/tests/sqllogictests/suites/base/03_common/03_0018_insert_into_variant +++ b/tests/sqllogictests/suites/base/03_common/03_0018_insert_into_variant @@ -13,10 +13,10 @@ CREATE TABLE IF NOT EXISTS t1(id Int null, var Variant null) Engine = Fuse statement ok INSERT INTO t1 (id, var) VALUES(1, null), (2, true),(3, false),(4, 1),(5, -1),(6, 1000),(7, -1000),(8, 9223372036854775807),(9, -9223372036854775808),(10, 18446744073709551615),(11, 0.12345679),(12, 0.12345678912121212),(13, '"abcd"'),(14, '[1,2,3]'),(15, '{"k":"v"}'),(16, '"a\"b\\"c"'),(17, '"http:\/\/databend.rs"') -statement error (?s)1001.*expected value, pos 1 while evaluating function `parse_json\('abc'\)` +statement error (?s)1006.*expected value, pos 1 while evaluating function `parse_json\('abc'\)` INSERT INTO t1 (id, var) VALUES(1, 'abc') -statement error (?s)1001.*EOF while parsing a value, pos 5 while evaluating function `parse_json\('\{"a":'\)` +statement error (?s)1006.*EOF while parsing a value, pos 5 while evaluating function `parse_json\('\{"a":'\)` INSERT INTO t1 (id, var) VALUES(1, '{"a":') statement error (?s)1046.*unable to cast type `Array\(UInt8\)` to type `Variant NULL` diff --git a/tests/sqllogictests/suites/base/03_common/03_0023_insert_into_array b/tests/sqllogictests/suites/base/03_common/03_0023_insert_into_array index ade37d943e11..c311322f2719 100644 --- a/tests/sqllogictests/suites/base/03_common/03_0023_insert_into_array +++ b/tests/sqllogictests/suites/base/03_common/03_0023_insert_into_array @@ -257,7 +257,7 @@ CREATE TABLE t12(id Int, arr Array(Date)) Engine = Fuse statement ok INSERT INTO t12 (id, arr) VALUES(1, ['2021-01-01', '2022-01-01']), (2, ['1990-12-01', '2030-01-12']) -statement error (?s)1001.*cannot parse to type `DATE` while evaluating function `to_date\('1000000\-01\-01'\)` +statement error (?s)1006.*cannot parse to type `DATE` while evaluating function `to_date\('1000000\-01\-01'\)` INSERT INTO t12 (id, arr) VALUES(3, ['1000000-01-01', '2000000-01-01']) query IT @@ -281,7 +281,7 @@ CREATE TABLE t13(id Int, arr Array(Timestamp)) Engine = Fuse statement ok INSERT INTO t13 (id, arr) VALUES(1, ['2021-01-01 01:01:01', '2022-01-01 01:01:01']), (2, ['1990-12-01 10:11:12', '2030-01-12 22:00:00']) -statement error (?s)1001.*cannot parse to type `TIMESTAMP` while evaluating function `to_timestamp\('1000000\-01\-01 01:01:01'\)` +statement error (?s)1006.*cannot parse to type `TIMESTAMP` while evaluating function `to_timestamp\('1000000\-01\-01 01:01:01'\)` INSERT INTO t13 (id, arr) VALUES(3, ['1000000-01-01 01:01:01', '2000000-01-01 01:01:01']) query IT diff --git a/tests/sqllogictests/suites/base/03_common/03_0027_insert_default b/tests/sqllogictests/suites/base/03_common/03_0027_insert_default index 09e2898d3de7..16259a121505 100644 --- a/tests/sqllogictests/suites/base/03_common/03_0027_insert_default +++ b/tests/sqllogictests/suites/base/03_common/03_0027_insert_default @@ -27,7 +27,7 @@ NULL 2 0.0 d statement ok drop table t_insert_default -statement error 1001 +statement error 1006 create table t_insert_default(a int null, b int not null default 2, c float not null, d varchar not null default 'd', e smallint not null default -872582066); diff --git a/tests/sqllogictests/suites/base/03_common/03_0037_insert_into_map b/tests/sqllogictests/suites/base/03_common/03_0037_insert_into_map index c5971a0bd594..83b1fd15ce31 100644 --- a/tests/sqllogictests/suites/base/03_common/03_0037_insert_into_map +++ b/tests/sqllogictests/suites/base/03_common/03_0037_insert_into_map @@ -32,7 +32,7 @@ select * from t1 where m[300] = '123' ---- 2 {300:'123'} -statement error 1001 +statement error 1006 INSERT INTO t1 (id, m) VALUES(1, {100:'k1',100:'k2'}) statement ok @@ -79,7 +79,7 @@ select {'id':number} from numbers(5) {'id':3} {'id':4} -statement error 1001 +statement error 1006 CREATE TABLE IF NOT EXISTS t3(id Int, m Map(Array(Date), String)) Engine = Fuse statement ok diff --git a/tests/sqllogictests/suites/base/11_data_type/11_0001_data_type_date_time b/tests/sqllogictests/suites/base/11_data_type/11_0001_data_type_date_time index bddf0d606975..2e3e63109430 100644 --- a/tests/sqllogictests/suites/base/11_data_type/11_0001_data_type_date_time +++ b/tests/sqllogictests/suites/base/11_data_type/11_0001_data_type_date_time @@ -249,28 +249,28 @@ select * from t order by id 2 2022-02-03 11:00:00.000000 3 2022-02-03 11:02:01.123456 -statement error 1001 +statement error 1006 insert into t values(1, '2022-02-03 003:02:02') -statement error 1001 +statement error 1006 insert into t values(1, '2022-02-03 03:002:02') -statement error 1001 +statement error 1006 insert into t values(1, '2022-02-03T03:02:000') -statement error 1001 +statement error 1006 insert into t values(1, '2022-02-03 03:02:000') -statement error 1001 +statement error 1006 insert into t values(1, '2022-02-03 03:02:00Z.123') -statement error 1001 +statement error 1006 insert into t values(1, '2022-02-03 03:02:00Z+08:00') -statement error 1001 +statement error 1006 insert into t values(1, '2022-02-03 03:02:00+0800') -statement error 1001 +statement error 1006 insert into t values(1, '2022-02-03T03:02:00+0800') statement ok diff --git a/tests/sqllogictests/suites/base/11_data_type/11_0006_data_type_decimal b/tests/sqllogictests/suites/base/11_data_type/11_0006_data_type_decimal index 061dc3cf2564..b86ba5f5544b 100644 --- a/tests/sqllogictests/suites/base/11_data_type/11_0006_data_type_decimal +++ b/tests/sqllogictests/suites/base/11_data_type/11_0006_data_type_decimal @@ -91,7 +91,7 @@ select 1::Decimal(41, 3)::Float64, 2::Decimal(42, 4)::Float64, 3::Decimal(55, 5) 1.0 2.0 3.0 10.0 ## to_decimal(decimal) overflow -statement error 1001 +statement error 1006 select 10000::decimal(5,0)::decimal(3,0); ## parser overflow @@ -100,7 +100,7 @@ select 2::Decimal(2000, 3); ## runtime overflow -statement error 1001 +statement error 1006 select 20000::Decimal(4, 3); query I @@ -108,7 +108,7 @@ select '010.010'::decimal(6,4); ---- 10.0100 -statement error 1001 +statement error 1006 select '010.010'::decimal(5,4); ## tests from chatgpt @@ -490,7 +490,7 @@ select count(*) from t_decimal_4_2; 0 ## overflow -statement error 1001 +statement error 1006 insert into t_decimal_4_2 values (11111); statement ok @@ -501,7 +501,7 @@ select count(*) from t_decimal_4_2; ---- 0 -statement error 1001 +statement error 1006 insert into t_decimal_4_2 values (1e4); query I @@ -988,16 +988,16 @@ statement ok drop table t -statement error 1001 +statement error 1006 select to_decimal(2,1)(10.1); -statement error 1001 +statement error 1006 select to_decimal(2,1)(10.11); statement ok create table if not exists test_decimal_insert(num DECIMAL(2,1)); -statement error 1001 +statement error 1006 insert into test_decimal_insert values(10.9); statement ok diff --git a/tests/sqllogictests/suites/crdb/and_or b/tests/sqllogictests/suites/crdb/and_or index bfdd3e29046c..4ebc25035f90 100644 --- a/tests/sqllogictests/suites/crdb/and_or +++ b/tests/sqllogictests/suites/crdb/and_or @@ -7,7 +7,7 @@ CREATE TABLE t (k INT null, a INT null, b INT null) statement ok INSERT INTO t VALUES (1, NULL, NULL), (2, NULL, 1), (3, 1, NULL), (4, 2, 0), (5, 3, 3) -statement error (?s)1001.*divided by zero while evaluating function `divide\(3, 0\)` +statement error (?s)1006.*divided by zero while evaluating function `divide\(3, 0\)` SELECT a <> 2 AND 3 / b = 1 FROM t ORDER BY k query I @@ -15,10 +15,10 @@ SELECT a FROM t WHERE a <> 2 AND 3 / b = 1 ORDER BY k ---- 3 -statement error (?s)1001.*divided by zero while evaluating function `divide\(3, 0\)` +statement error (?s)1006.*divided by zero while evaluating function `divide\(3, 0\)` SELECT a = 2 OR 3 / b = 1 FROM t ORDER BY k -statement error (?s)1001.*divided by zero while evaluating function `divide\(3, 0\)` +statement error (?s)1006.*divided by zero while evaluating function `divide\(3, 0\)` SELECT a FROM t WHERE a = 2 OR 3 / b = 1 ORDER BY k statement ok diff --git a/tests/sqllogictests/suites/crdb/overflow b/tests/sqllogictests/suites/crdb/overflow index aa14d21932db..90d4b47bc973 100644 --- a/tests/sqllogictests/suites/crdb/overflow +++ b/tests/sqllogictests/suites/crdb/overflow @@ -4,7 +4,7 @@ drop table if exists large_numbers statement ok CREATE TABLE large_numbers (a INT8) -statement error 1001 +statement error 1006 INSERT INTO large_numbers VALUES (9223372036854775807),(1) query I @@ -15,7 +15,7 @@ NULL statement ok DELETE FROM large_numbers -statement error 1001 +statement error 1006 INSERT INTO large_numbers VALUES (-9223372036854775808),(-1) query I diff --git a/tests/sqllogictests/suites/duckdb/common/test_cast_hugeint.test b/tests/sqllogictests/suites/duckdb/common/test_cast_hugeint.test index a1c7c7a8430e..cb15fb66f8db 100644 --- a/tests/sqllogictests/suites/duckdb/common/test_cast_hugeint.test +++ b/tests/sqllogictests/suites/duckdb/common/test_cast_hugeint.test @@ -174,14 +174,14 @@ select '4294967295'::BIGINT::UINT32 # ---- # 4294967295 -statement error 1001 +statement error 1006 select '18446744073709551615'::BIGINT::UINT64 # TODO https://github.com/datafuselabs/databend/issues/7347 # statement error XXX # select '-1'::BIGINT::UINT64 -statement error 1001 +statement error 1006 select '18446744073709551616'::BIGINT::UINT64 query I diff --git a/tests/sqllogictests/suites/duckdb/issues/monetdb/test_null_matches_outer.test b/tests/sqllogictests/suites/duckdb/issues/monetdb/test_null_matches_outer.test index 2f27811d4480..331b7152c283 100644 --- a/tests/sqllogictests/suites/duckdb/issues/monetdb/test_null_matches_outer.test +++ b/tests/sqllogictests/suites/duckdb/issues/monetdb/test_null_matches_outer.test @@ -16,7 +16,7 @@ insert into "E" values (1, 'one') statement ok insert into "E" values (2, 'two') -statement error 1001 +statement error 1006 insert into "E" values (null, null) statement ok @@ -28,7 +28,7 @@ insert into "I" values (2, 'due') statement ok insert into "I" values (4, 'quattro') -statement error 1001 +statement error 1006 insert into "I" values (null, 'this is not null') query ITTT diff --git a/tests/sqllogictests/suites/duckdb/issues/rigger/complex_division.test b/tests/sqllogictests/suites/duckdb/issues/rigger/complex_division.test index 5b31650547da..b80a10de9876 100644 --- a/tests/sqllogictests/suites/duckdb/issues/rigger/complex_division.test +++ b/tests/sqllogictests/suites/duckdb/issues/rigger/complex_division.test @@ -28,5 +28,5 @@ SELECT c0, c1 FROM t0 order by c1 0.5 NULL 0.5 NULL -statement error 1001 +statement error 1006 SELECT c0, c1, ((t0.c0)/(t0.c1)) FROM t0 order by c0, c1 diff --git a/tests/sqllogictests/suites/duckdb/issues/rigger/division_by_zero_error.test b/tests/sqllogictests/suites/duckdb/issues/rigger/division_by_zero_error.test index bcb30e3afc9f..8999bef54d07 100644 --- a/tests/sqllogictests/suites/duckdb/issues/rigger/division_by_zero_error.test +++ b/tests/sqllogictests/suites/duckdb/issues/rigger/division_by_zero_error.test @@ -13,10 +13,10 @@ INSERT INTO t0 VALUES(0.0,0.0) statement ok INSERT INTO t0 VALUES(0.0,NULL) -statement error 1001 +statement error 1006 SELECT t0.c1, (1/t0.c1) IS NULL FROM t0 order by c1 -statement error 1001 +statement error 1006 SELECT t0.c1 FROM t0 WHERE (((1/(t0.c1))) IS NULL) order by c1 statement ok diff --git a/tests/sqllogictests/suites/duckdb/sql/aggregate/aggregates/test_avg.test b/tests/sqllogictests/suites/duckdb/sql/aggregate/aggregates/test_avg.test index 4f3365608df7..2e3443b51f64 100644 --- a/tests/sqllogictests/suites/duckdb/sql/aggregate/aggregates/test_avg.test +++ b/tests/sqllogictests/suites/duckdb/sql/aggregate/aggregates/test_avg.test @@ -15,10 +15,10 @@ SELECT AVG(3), AVG(NULL) ---- 3.0 NULL -statement error 1001 +statement error 1006 SELECT AVG(3::SMALLINT), AVG(NULL::SMALLINT) -statement error 1001 +statement error 1006 SELECT AVG(3::DOUBLE), AVG(NULL::DOUBLE) statement error 1005 diff --git a/tests/sqllogictests/suites/duckdb/sql/cast/test_exponent_in_cast.test b/tests/sqllogictests/suites/duckdb/sql/cast/test_exponent_in_cast.test index 2e73e158f37f..45cba0840a35 100644 --- a/tests/sqllogictests/suites/duckdb/sql/cast/test_exponent_in_cast.test +++ b/tests/sqllogictests/suites/duckdb/sql/cast/test_exponent_in_cast.test @@ -1,22 +1,22 @@ -statement error 1001 +statement error 1006 SELECT CAST('e1' AS INTEGER) -statement error 1001 +statement error 1006 SELECT CAST(' e1' AS INTEGER) -statement error 1001 +statement error 1006 SELECT CAST(' E1' AS INTEGER) -statement error 1001 +statement error 1006 SELECT CAST('e1' AS DOUBLE) -statement error 1001 +statement error 1006 SELECT CAST(' e1' AS DOUBLE) -statement error 1001 +statement error 1006 SELECT CAST(' E1' AS DOUBLE) -statement error 1001 +statement error 1006 SELECT CAST(' 1e1' AS INTEGER) query F diff --git a/tests/sqllogictests/suites/duckdb/sql/cast/test_try_cast.test b/tests/sqllogictests/suites/duckdb/sql/cast/test_try_cast.test index 2ddd3aa1ad72..a73d5d587df3 100644 --- a/tests/sqllogictests/suites/duckdb/sql/cast/test_try_cast.test +++ b/tests/sqllogictests/suites/duckdb/sql/cast/test_try_cast.test @@ -6,7 +6,7 @@ SELECT TRY_CAST('hello' as INTEGER) ---- NULL -statement error 1001 +statement error 1006 SELECT CAST('hello' as INTEGER) query IIII diff --git a/tests/sqllogictests/suites/query/02_function/02_0000_function_arithmetic b/tests/sqllogictests/suites/query/02_function/02_0000_function_arithmetic index 2be9b8e8b6b4..09bc2c086a55 100644 --- a/tests/sqllogictests/suites/query/02_function/02_0000_function_arithmetic +++ b/tests/sqllogictests/suites/query/02_function/02_0000_function_arithmetic @@ -13,7 +13,7 @@ SELECT 8 / 2 ---- 4.0 -statement error 1001 +statement error 1006 SELECT 8 / 0 query III @@ -21,10 +21,10 @@ SELECT 1 // 2, 4 // 3, 1 // (5 // 2) ---- 0 1 0 -statement error 1001 +statement error 1006 SELECT 8 // 0 -statement error 1001 +statement error 1006 SELECT 20 DIV 0 query I diff --git a/tests/sqllogictests/suites/query/02_function/02_0002_function_cast b/tests/sqllogictests/suites/query/02_function/02_0002_function_cast index c441b6ec021b..bb605bb34af7 100644 --- a/tests/sqllogictests/suites/query/02_function/02_0002_function_cast +++ b/tests/sqllogictests/suites/query/02_function/02_0002_function_cast @@ -64,15 +64,15 @@ SELECT CAST(CAST(1 + 1 + 1, String) AS Int8) 3 -statement error 1001 +statement error 1006 SELECT CAST(Null as Int64) -statement error 1001 +statement error 1006 SELECT CAST(Null as Varchar) -statement error 1001 +statement error 1006 SELECT CAST(Null as Boolean) query B @@ -87,23 +87,23 @@ SELECT CAST('33' as unsigned) = 33 -statement error 1001 +statement error 1006 SELECT CAST('-33aa' as signed) = 33 -statement error 1001 +statement error 1006 SELECT CAST('33 aa' as unsigned) = 33 -statement error 1001 +statement error 1006 SELECT CAST('-33' as unsigned) = 0 -statement error 1001 +statement error 1006 SELECT CAST('aa' as unsigned) = 0 -statement error 1001 +statement error 1006 SELECT CAST('aa' as Float64) = 0 query B @@ -119,7 +119,7 @@ SELECT '33'::unsigned = 33 -statement error 1001 +statement error 1006 SELECT '-33aa'::signed = 33 query B @@ -217,15 +217,15 @@ SELECT parse_json('"false"')::boolean 0 -statement error 1001 +statement error 1006 SELECT parse_json('"test"')::boolean -statement error 1001 +statement error 1006 SELECT to_variant(1)::boolean -statement error 1001 +statement error 1006 SELECT parse_json('null')::boolean query I @@ -349,15 +349,15 @@ SELECT parse_json('"9223372036854775807"')::int64 9223372036854775807 -statement error 1001 +statement error 1006 SELECT parse_json('"test"')::uint64 -statement error 1001 +statement error 1006 SELECT parse_json('"test"')::int64 -statement error 1001 +statement error 1006 SELECT parse_json('null')::int64 query F @@ -381,15 +381,15 @@ SELECT parse_json('"1234.5678"')::float64 1234.5678 -statement error 1001 +statement error 1006 SELECT parse_json('"test"')::float32 -statement error 1001 +statement error 1006 SELECT parse_json('"test"')::float64 -statement error 1001 +statement error 1006 SELECT parse_json('null')::float64 query T @@ -403,15 +403,15 @@ SELECT parse_json('"2022-01-01 01:01:01"')::datetime 2022-01-01 01:01:01.000000 -statement error 1001 +statement error 1006 SELECT parse_json('"test"')::date -statement error 1001 +statement error 1006 SELECT parse_json('"test"')::datetime -statement error 1001 +statement error 1006 SELECT parse_json('null')::datetime query T diff --git a/tests/sqllogictests/suites/query/02_function/02_0002_function_others b/tests/sqllogictests/suites/query/02_function/02_0002_function_others index 222e8c666e82..aab56c2038da 100644 --- a/tests/sqllogictests/suites/query/02_function/02_0002_function_others +++ b/tests/sqllogictests/suites/query/02_function/02_0002_function_others @@ -34,7 +34,7 @@ SELECT INET_ATON(NULL) NULL -statement error 1001 +statement error 1006 SELECT INET_ATON('hello') query T diff --git a/tests/sqllogictests/suites/query/02_function/02_0010_function_if b/tests/sqllogictests/suites/query/02_function/02_0010_function_if index 61258ddd5730..801edaaad7e9 100644 --- a/tests/sqllogictests/suites/query/02_function/02_0010_function_if +++ b/tests/sqllogictests/suites/query/02_function/02_0010_function_if @@ -48,7 +48,7 @@ select if(number>4, number / 0, number) from numbers(3) order by number 1.0 2.0 -statement error 1001 +statement error 1006 select if(number<4, number / 0, number) from numbers(3) order by number query F diff --git a/tests/sqllogictests/suites/query/02_function/02_0012_function_datetimes b/tests/sqllogictests/suites/query/02_function/02_0012_function_datetimes index 9f9607c8c2e6..a365e8482938 100644 --- a/tests/sqllogictests/suites/query/02_function/02_0012_function_datetimes +++ b/tests/sqllogictests/suites/query/02_function/02_0012_function_datetimes @@ -68,7 +68,7 @@ select to_date('9999-12-31') 9999-12-31 -statement error 1001 +statement error 1006 select to_date('10000-12-31') query T @@ -127,19 +127,19 @@ select to_unix_timestamp('2022-12-31T23:59:59-08:00') 1672559999 -statement error 1001 +statement error 1006 select to_datetime('9999-01-01 00x') -statement error 1001 +statement error 1006 select to_datetime('9999-01-01 001') -statement error 1001 +statement error 1006 select to_datetime('9999-01-01 01:123') -statement error 1001 +statement error 1006 select to_datetime('9999-01-01 01:12.123+02:00') -statement error 1001 +statement error 1006 select to_datetime('10000-01-01 00:00:00') query T @@ -222,10 +222,10 @@ select to_date('2023-01-01') - 100 = to_date('2022-09-23') ---- 1 -statement error 1001 +statement error 1006 select to_date('2023-01-01') + 100000000 -statement error 1001 +statement error 1006 select to_date('2023-01-01') - 100000000 @@ -578,10 +578,10 @@ select add_years(to_datetime(1582970400000000), cast(-50, INT8)) 1970-02-28 10:00:00.000000 -statement error 1001 +statement error 1006 select add_years(to_date('9999-12-31'), 1) -statement error 1001 +statement error 1006 select add_years(to_datetime('9999-12-31 23:59:59'), 1) # 2020-2-29 - 13 months @@ -597,10 +597,10 @@ select subtract_months(to_datetime(1582970400000000), cast(122, INT16)) 2009-12-29 10:00:00.000000 -statement error 1001 +statement error 1006 select subtract_months(to_date('1000-01-01'), 1) -statement error 1001 +statement error 1006 select subtract_months(to_datetime('1000-01-01 00:00:00'), 1) # 2020-2-29 + 1 day @@ -615,10 +615,10 @@ select add_days(to_datetime(1582970400000000), cast(-1, INT16)) ---- 2020-02-28 10:00:00.000000 -statement error 1001 +statement error 1006 select add_days(to_date('9999-12-31'), 1) -statement error 1001 +statement error 1006 select add_days(to_datetime('9999-12-31 23:59:59'), 1) # 2020-2-29T10:00:00 + 25 hours @@ -634,10 +634,10 @@ select add_hours(to_date(18321), 1) 2020-02-29 01:00:00.000000 -statement error 1001 +statement error 1006 select add_hours(to_date('9999-12-31'), 24) -statement error 1001 +statement error 1006 select add_hours(to_datetime('9999-12-31 23:59:59'), 1) # 2020-2-29T10:00:00 - 1 minutes @@ -646,10 +646,10 @@ select subtract_minutes(to_datetime(1582970400000000), cast(1, INT32)) ---- 2020-02-29 09:59:00.000000 -statement error 1001 +statement error 1006 select subtract_minutes(to_date('1000-01-01'), 1) -statement error 1001 +statement error 1006 select subtract_minutes(to_datetime('1000-01-01 00:00:00'), 1) # 2020-2-29T10:00:00 + 61 seconds @@ -668,10 +668,10 @@ select to_datetime('2023-01-01 00:00:00') - 10000000000 ---- 2022-12-31 21:13:20.000000 -statement error 1001 +statement error 1006 select to_datetime('2023-01-01 00:00:00') + 1000000000000000000 -statement error 1001 +statement error 1006 select to_datetime('2023-01-01 00:00:00') - 1000000000000000000 query I @@ -924,7 +924,7 @@ insert into t values('2022-04-02 15:10:28.221', '2022-04-02 15:10:28.221', '9999 statement ok insert into t values('0999-04-02 15:10:28.221', '2022-04-02 15:10:28.222', '2020-10-10') -statement error 1001 +statement error 1006 insert into t values('10000-01-01 00:00:00', '2022-04-02 15:10:28.221', '2020-10-10') statement ok @@ -937,7 +937,7 @@ statement ok insert into t values('2022-04-02T15:10:28-08:13', '2022-04-02T15:10:28.223+08:00', '2022-04-02') skipif clickhouse -statement error 1001 +statement error 1006 insert into t values('2022-04-02 15:10:28.221', '2022-04-02 15:10:28.221', '10000-10-10') query TTT @@ -968,10 +968,10 @@ select to_date('精彩的2022年,美丽的02month,激动の02d', '精彩的%Y ---- 2022-02-02 -statement error 1001 +statement error 1006 select date_format('', ''); -statement error 1001 +statement error 1006 select date_format('2022-2-04T03:58:59', '%Y年%m月%d日,%H时%M分%S秒'); query T diff --git a/tests/sqllogictests/suites/query/02_function/02_0012_function_datetimes_tz b/tests/sqllogictests/suites/query/02_function/02_0012_function_datetimes_tz index 765c124cbf74..711a56096457 100644 --- a/tests/sqllogictests/suites/query/02_function/02_0012_function_datetimes_tz +++ b/tests/sqllogictests/suites/query/02_function/02_0012_function_datetimes_tz @@ -397,11 +397,11 @@ select count_if(y = true) from (select to_timestamp(to_date(number)) as ts, to_ statement ok set timezone='Europe/London'; -statement error 1001 +statement error 1006 ---- select to_date(to_timestamp('2021-03-28 01:00')); -statement error 1001 +statement error 1006 ---- select '2021-03-28 01:59:59'::timestamp; @@ -413,7 +413,7 @@ select to_date('1941-03-15'); ---- 1941-03-15 -statement error 1001 +statement error 1006 select to_date('1941-03-15 00:00:00'); query T diff --git a/tests/sqllogictests/suites/query/02_function/02_0013_function_running_difference b/tests/sqllogictests/suites/query/02_function/02_0013_function_running_difference index 71fa6beae616..efa6ef8a08ea 100644 --- a/tests/sqllogictests/suites/query/02_function/02_0013_function_running_difference +++ b/tests/sqllogictests/suites/query/02_function/02_0013_function_running_difference @@ -15,7 +15,7 @@ select running_difference(a), running_difference(b), running_difference(c), runn 2 2 2 2.0 0 5 5 5 5.0 0 -statement error 1001 +statement error 1006 select d, running_difference(d) from running_difference_test statement ok diff --git a/tests/sqllogictests/suites/query/02_function/02_0014_function_maths b/tests/sqllogictests/suites/query/02_function/02_0014_function_maths index e88a7c4cb3f7..d2780f51f7fd 100644 --- a/tests/sqllogictests/suites/query/02_function/02_0014_function_maths +++ b/tests/sqllogictests/suites/query/02_function/02_0014_function_maths @@ -264,7 +264,7 @@ SELECT cbrt(-64) ---- -4.0 -statement error 1001 +statement error 1006 SELECT sqrt('a') query F @@ -314,23 +314,23 @@ SELECT factorial(5) ---- 120 -statement error 1001 +statement error 1006 SELECT factorial(30) -statement error 1001 +statement error 1006 SELECT pow('a', 2) -statement error 1001 +statement error 1006 SELECT pow(2, 'a') -statement error 1001 +statement error 1006 SELECT abs('-233.0') -statement error 1001 +statement error 1006 SELECT abs('blah') diff --git a/tests/sqllogictests/suites/query/02_function/02_0022_function_strings_unhex b/tests/sqllogictests/suites/query/02_function/02_0022_function_strings_unhex index 0e5cbe9d8f2a..9a1b8dc952af 100644 --- a/tests/sqllogictests/suites/query/02_function/02_0022_function_strings_unhex +++ b/tests/sqllogictests/suites/query/02_function/02_0022_function_strings_unhex @@ -4,7 +4,7 @@ select unhex('616263') abc -statement error 1001 +statement error 1006 select unhex('hello') query T diff --git a/tests/sqllogictests/suites/query/02_function/02_0024_function_strings_base_64 b/tests/sqllogictests/suites/query/02_function/02_0024_function_strings_base_64 index 96dc8a734aeb..132fc2a821e6 100644 --- a/tests/sqllogictests/suites/query/02_function/02_0024_function_strings_base_64 +++ b/tests/sqllogictests/suites/query/02_function/02_0024_function_strings_base_64 @@ -19,7 +19,7 @@ SELECT FROM_BASE64(NULL) NULL -statement error 1001 +statement error 1006 SELECT FROM_BASE64('1') query T diff --git a/tests/sqllogictests/suites/query/02_function/02_0034_function_strings_space b/tests/sqllogictests/suites/query/02_function/02_0034_function_strings_space index f0e6f994ef4c..9822fab6e0dc 100644 --- a/tests/sqllogictests/suites/query/02_function/02_0034_function_strings_space +++ b/tests/sqllogictests/suites/query/02_function/02_0034_function_strings_space @@ -12,5 +12,5 @@ SELECT SPACE(1) statement ok SELECT SPACE(2) -statement error 1001 +statement error 1006 SELECT SPACE(2000000) diff --git a/tests/sqllogictests/suites/query/02_function/02_0035_function_strings_pad b/tests/sqllogictests/suites/query/02_function/02_0035_function_strings_pad index 0601aa2e87c9..940c8724f8fb 100644 --- a/tests/sqllogictests/suites/query/02_function/02_0035_function_strings_pad +++ b/tests/sqllogictests/suites/query/02_function/02_0035_function_strings_pad @@ -36,13 +36,13 @@ NULL statement ok SELECT LPAD('', 0, '') -statement error 1001 +statement error 1006 SELECT LPAD('', 10, '') -statement error 1001 +statement error 1006 SELECT LPAD('123', 10, '') -statement error 1001 +statement error 1006 SELECT LPAD('123', 2000000, '') query T @@ -177,13 +177,13 @@ NULL statement ok SELECT RPAD('', 0, '') -statement error 1001 +statement error 1006 SELECT RPAD('', 10, '') -statement error 1001 +statement error 1006 SELECT RPAD('123', 10, '') -statement error 1001 +statement error 1006 SELECT RPAD('123', 2000000, '') query T diff --git a/tests/sqllogictests/suites/query/02_function/02_0048_function_semi_structureds_parse_json b/tests/sqllogictests/suites/query/02_function/02_0048_function_semi_structureds_parse_json index 68e85026f7d4..a12505c70aab 100644 --- a/tests/sqllogictests/suites/query/02_function/02_0048_function_semi_structureds_parse_json +++ b/tests/sqllogictests/suites/query/02_function/02_0048_function_semi_structureds_parse_json @@ -64,11 +64,11 @@ select parse_json('{ "test" : "\\"abc\"测试⚠️✅❌မြန်မာဘ ---- {"test":"\"abc\"测试⚠️✅❌မြန်မာဘာသာ"} -statement error 1001 +statement error 1006 select parse_json('[1,') -statement error 1001 +statement error 1006 select parse_json('"ab') diff --git a/tests/sqllogictests/suites/query/02_function/02_0051_function_semi_structureds_get b/tests/sqllogictests/suites/query/02_function/02_0051_function_semi_structureds_get index 58e462c1bc65..f8eb292044bf 100644 --- a/tests/sqllogictests/suites/query/02_function/02_0051_function_semi_structureds_get +++ b/tests/sqllogictests/suites/query/02_function/02_0051_function_semi_structureds_get @@ -320,7 +320,7 @@ select id, get(arr, 5) from t4 2 NULL -statement error 1001 +statement error 1006 select id, get(arr, 'a') from t4 query ITS @@ -365,7 +365,7 @@ query IT select id, json_path_query(obj, '$.b?(@.c > 2)') from t2 ---- -statement error 1001 +statement error 1006 select id, json_path_query(obj, '--') from t2 query IT @@ -403,7 +403,7 @@ select id, json_path_query_array(obj, '$.b?(@.c > 2)') from t2 ---- 1 [] -statement error 1001 +statement error 1006 select id, json_path_query_array(obj, '--') from t2 query IT @@ -441,7 +441,7 @@ select id, json_path_query_first(obj, '$.b?(@.c > 2)') from t2 ---- 1 NULL -statement error 1001 +statement error 1006 select id, json_path_query_first(obj, '--') from t2 query T diff --git a/tests/sqllogictests/suites/query/02_function/02_0052_function_string_regexp_substr b/tests/sqllogictests/suites/query/02_function/02_0052_function_string_regexp_substr index dde68293d6ea..911f67dbc1e8 100644 --- a/tests/sqllogictests/suites/query/02_function/02_0052_function_string_regexp_substr +++ b/tests/sqllogictests/suites/query/02_function/02_0052_function_string_regexp_substr @@ -90,7 +90,7 @@ CREATE TABLE tt2(b int not null) Engine = Fuse statement ok INSERT INTO tt2 VALUES (-1448403955),(1978682043),(775979323),(-2057711809),(183393966),(618200641) -statement error 1001 +statement error 1006 SELECT string_agg(regexp_substr(tt1.d, '-', tt1.c)), tt1.a FROM tt1 FULL OUTER JOIN tt2 USING(b) GROUP BY ROLLUP (tt1.a) diff --git a/tests/sqllogictests/suites/query/02_function/02_0054_function_string_regexp_replace b/tests/sqllogictests/suites/query/02_function/02_0054_function_string_regexp_replace index 95de622e7e80..7558deb8cc8f 100644 --- a/tests/sqllogictests/suites/query/02_function/02_0054_function_string_regexp_replace +++ b/tests/sqllogictests/suites/query/02_function/02_0054_function_string_regexp_replace @@ -102,13 +102,13 @@ statement ok DROP TABLE t1 -statement error 1001 +statement error 1006 SELECT REGEXP_REPLACE('a b c', 'b', 'X', 0) -statement error 1001 +statement error 1006 SELECT REGEXP_REPLACE('a b c', 'b', 'X', 1, -1) -statement error 1001 +statement error 1006 SELECT REGEXP_REPLACE('a b c', 'b', 'X', 1, 0, '-i') diff --git a/tests/sqllogictests/suites/query/02_function/02_0065_function_json b/tests/sqllogictests/suites/query/02_function/02_0065_function_json index a3c4c95e2d08..a53351f6d3d2 100644 --- a/tests/sqllogictests/suites/query/02_function/02_0065_function_json +++ b/tests/sqllogictests/suites/query/02_function/02_0065_function_json @@ -73,7 +73,7 @@ SELECT json_object('k1', 1, 'k2', 'str', 'k3', [1,2], 'k4', {'k':'v'}, 'k5', nul ---- {"k1":1,"k2":"str","k3":[1,2],"k4":{"k":"v"}} -statement error 1001 +statement error 1006 SELECT json_object('k1', 1, 'k2', 'str', 'k3') query T @@ -96,7 +96,7 @@ SELECT json_object_keep_null('k1', 1, 'k2', 'str', 'k3', [1,2], 'k4', {'k':'v'}, ---- {"k1":1,"k2":"str","k3":[1,2],"k4":{"k":"v"},"k5":null} -statement error 1001 +statement error 1006 SELECT json_object_keep_null('k1', 1, 'k2', 'str', 'k3') query T diff --git a/tests/sqllogictests/suites/query/functions/auto_parse.test b/tests/sqllogictests/suites/query/functions/auto_parse.test index 0be2aece2c33..749101831325 100644 --- a/tests/sqllogictests/suites/query/functions/auto_parse.test +++ b/tests/sqllogictests/suites/query/functions/auto_parse.test @@ -3,7 +3,7 @@ SELECT '1' + 1 ---- 2 -statement error 1001 +statement error 1006 SELECT 'foo' + 1 ---- diff --git a/tests/suites/0_stateless/17_altertable/17_0005_alter_table_modify_column_type.result b/tests/suites/0_stateless/17_altertable/17_0005_alter_table_modify_column_type.result index c4cd645e7c14..ae53211f775a 100644 --- a/tests/suites/0_stateless/17_altertable/17_0005_alter_table_modify_column_type.result +++ b/tests/suites/0_stateless/17_altertable/17_0005_alter_table_modify_column_type.result @@ -6,10 +6,10 @@ c INT NO 0 a FLOAT NO 0 b VARCHAR NO '' c INT NO 0 -ERROR 1105 (HY000) at line 1: Internal. Code: 1001, Text = invalid float literal while evaluating function `to_float32('a')`. +ERROR 1105 (HY000) at line 1: BadArguments. Code: 1006, Text = invalid float literal while evaluating function `to_float32('a')`. ERROR 1105 (HY000) at line 1: UnknownColumn. Code: 1058, Text = Cannot find column b. 0 1 -ERROR 1105 (HY000) at line 1: Internal. Code: 1001, Text = error: +ERROR 1105 (HY000) at line 1: BadArguments. Code: 1006, Text = error: --> SQL:1:1 | 1 | alter table test_modify_column_type.c modify column a float not null default 'a' diff --git a/tests/suites/0_stateless/20+_others/20_0013_pretty_error.result b/tests/suites/0_stateless/20+_others/20_0013_pretty_error.result index 2dc5cfd0de17..a9e1d6928856 100644 --- a/tests/suites/0_stateless/20+_others/20_0013_pretty_error.result +++ b/tests/suites/0_stateless/20+_others/20_0013_pretty_error.result @@ -30,7 +30,7 @@ has tried possible overloads: to_base64(String NULL) :: String NULL : unable to unify `UInt8` with `String` . -ERROR 1105 (HY000) at line 1: Internal. Code: 1001, Text = error: +ERROR 1105 (HY000) at line 1: BadArguments. Code: 1006, Text = error: --> SQL:1:12 | 1 | select 1 + 'a' From 91bffcfb52ff9487f8bc12a1569a519436b14bc2 Mon Sep 17 00:00:00 2001 From: everpcpc Date: Fri, 20 Oct 2023 11:19:41 +0800 Subject: [PATCH 03/39] chore(ci): skip clean on local benchmark (#13355) --- .github/actions/benchmark_local/action.yml | 8 -------- .github/workflows/bindings.python.yml | 2 ++ .github/workflows/dev.yml | 1 + benchmark/clickbench/hits/clear.sql | 2 +- benchmark/clickbench/tpch/clear.sql | 16 ++++++++-------- 5 files changed, 12 insertions(+), 17 deletions(-) diff --git a/.github/actions/benchmark_local/action.yml b/.github/actions/benchmark_local/action.yml index f0e92eba318f..f540ca4f0e11 100644 --- a/.github/actions/benchmark_local/action.yml +++ b/.github/actions/benchmark_local/action.yml @@ -58,11 +58,3 @@ runs: with: name: benchmark-${{ inputs.dataset }} path: benchmark/clickbench/result-${{ inputs.dataset }}-local.json - - - name: Clean test data - working-directory: benchmark/clickbench - if: always() - continue-on-error: true - shell: bash - run: | - bendsql <"${{ inputs.dataset }}/clear.sql" diff --git a/.github/workflows/bindings.python.yml b/.github/workflows/bindings.python.yml index 752fe24eafb7..da9954a4ed2c 100644 --- a/.github/workflows/bindings.python.yml +++ b/.github/workflows/bindings.python.yml @@ -35,6 +35,7 @@ jobs: target: ${{ matrix.target }} version: ${{ inputs.tag }} - name: Publish to PyPI + timeout-minutes: 10 if: inputs.tag uses: pypa/gh-action-pypi-publish@release/v1 with: @@ -61,6 +62,7 @@ jobs: version: ${{ inputs.tag }} - name: Publish to PyPI if: inputs.tag + timeout-minutes: 10 env: MATURIN_PYPI_TOKEN: ${{ secrets.PYPI_PASSWORD }} uses: PyO3/maturin-action@v1 diff --git a/.github/workflows/dev.yml b/.github/workflows/dev.yml index 79ce331fe2bb..97ec49408ef4 100644 --- a/.github/workflows/dev.yml +++ b/.github/workflows/dev.yml @@ -32,6 +32,7 @@ jobs: docs/** website/** **.md + benchmark/** docker/** scripts/setup/** .devcontainer/** diff --git a/benchmark/clickbench/hits/clear.sql b/benchmark/clickbench/hits/clear.sql index e70c0347a8da..523476674517 100644 --- a/benchmark/clickbench/hits/clear.sql +++ b/benchmark/clickbench/hits/clear.sql @@ -1 +1 @@ -drop table hits all; +drop table if exists hits; diff --git a/benchmark/clickbench/tpch/clear.sql b/benchmark/clickbench/tpch/clear.sql index ded376e4a710..933245e2c6f1 100644 --- a/benchmark/clickbench/tpch/clear.sql +++ b/benchmark/clickbench/tpch/clear.sql @@ -1,8 +1,8 @@ -drop table customer all; -drop table lineitem all; -drop table nation all; -drop table orders all; -drop table partsupp all; -drop table part all; -drop table region all; -drop table supplier all; +drop table if exists customer; +drop table if exists lineitem; +drop table if exists nation; +drop table if exists orders; +drop table if exists partsupp; +drop table if exists part; +drop table if exists region; +drop table if exists supplier; From 3f2e4191eb469c16954f712e0e08930bea54d911 Mon Sep 17 00:00:00 2001 From: zhihanz Date: Fri, 20 Oct 2023 12:08:10 +0800 Subject: [PATCH 04/39] feat: add SQL syntax support for alter, execute, describe, show, drop task (#13344) --- src/query/ast/src/ast/statements/statement.rs | 14 +- src/query/ast/src/ast/statements/task.rs | 129 ++++++++++++++ src/query/ast/src/parser/statement.rs | 114 ++++++++++++ src/query/ast/src/parser/token.rs | 9 + src/query/ast/src/visitors/visitor.rs | 10 ++ src/query/ast/src/visitors/visitor_mut.rs | 10 ++ src/query/ast/src/visitors/walk.rs | 5 + src/query/ast/src/visitors/walk_mut.rs | 5 + src/query/ast/tests/it/parser.rs | 10 ++ .../ast/tests/it/testdata/statement-error.txt | 6 +- src/query/ast/tests/it/testdata/statement.txt | 163 +++++++++++++++++- .../interpreters/access/privilege_access.rs | 7 +- .../src/interpreters/interpreter_factory.rs | 5 + src/query/sql/src/planner/binder/binder.rs | 15 ++ src/query/sql/src/planner/binder/ddl/task.rs | 154 +++++++++++++++-- .../sql/src/planner/format/display_plan.rs | 5 + src/query/sql/src/planner/plans/ddl/task.rs | 65 +++++++ src/query/sql/src/planner/plans/plan.rs | 10 ++ 18 files changed, 710 insertions(+), 26 deletions(-) diff --git a/src/query/ast/src/ast/statements/statement.rs b/src/query/ast/src/ast/statements/statement.rs index 2bd44719af82..324b8709f86e 100644 --- a/src/query/ast/src/ast/statements/statement.rs +++ b/src/query/ast/src/ast/statements/statement.rs @@ -233,6 +233,11 @@ pub enum Statement { // tasks CreateTask(CreateTaskStmt), + AlterTask(AlterTaskStmt), + ExecuteTask(ExecuteTaskStmt), + DescribeTask(DescribeTaskStmt), + DropTask(DropTaskStmt), + ShowTasks(ShowTasksStmt), } #[derive(Debug, Clone, PartialEq)] @@ -530,9 +535,12 @@ impl Display for Statement { Statement::DropNetworkPolicy(stmt) => write!(f, "{stmt}")?, Statement::DescNetworkPolicy(stmt) => write!(f, "{stmt}")?, Statement::ShowNetworkPolicies => write!(f, "SHOW NETWORK POLICIES")?, - Statement::CreateTask(stmt) => { - write!(f, "{stmt}", stmt = stmt)?; - } + Statement::CreateTask(stmt) => write!(f, "{stmt}")?, + Statement::AlterTask(stmt) => write!(f, "{stmt}")?, + Statement::ExecuteTask(stmt) => write!(f, "{stmt}")?, + Statement::DropTask(stmt) => write!(f, "{stmt}")?, + Statement::ShowTasks(stmt) => write!(f, "{stmt}")?, + Statement::DescribeTask(stmt) => write!(f, "{stmt}")?, } Ok(()) } diff --git a/src/query/ast/src/ast/statements/task.rs b/src/query/ast/src/ast/statements/task.rs index 514f9bbaef64..3ea17c1510b7 100644 --- a/src/query/ast/src/ast/statements/task.rs +++ b/src/query/ast/src/ast/statements/task.rs @@ -15,6 +15,8 @@ use std::fmt::Display; use std::fmt::Formatter; +use crate::ast::ShowLimit; + #[derive(Debug, Clone, PartialEq, Eq)] pub struct CreateTaskStmt { pub if_not_exists: bool, @@ -46,6 +48,7 @@ impl Display for CreateTaskStmt { write!(f, " COMMENTS = '{}'", self.comments)?; } + write!(f, " AS {}", self.sql)?; Ok(()) } } @@ -86,3 +89,129 @@ impl Display for ScheduleOptions { } } } + +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct AlterTaskStmt { + pub if_exists: bool, + pub name: String, + pub options: AlterTaskOptions, +} +#[derive(Debug, Clone, PartialEq, Eq)] +pub enum AlterTaskOptions { + Resume, + Suspend, + Set { + warehouse: Option, + schedule: Option, + suspend_task_after_num_failures: Option, + comments: Option, + }, + Unset { + warehouse: bool, + }, + // Change SQL + ModifyAs(String), +} + +impl Display for AlterTaskOptions { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match self { + AlterTaskOptions::Resume => write!(f, " RESUME"), + AlterTaskOptions::Suspend => write!(f, " SUSPEND"), + AlterTaskOptions::Set { + warehouse, + schedule, + suspend_task_after_num_failures, + comments, + } => { + if let Some(wh) = warehouse { + write!(f, " SET WAREHOUSE = {}", wh)?; + } + if let Some(schedule) = schedule { + write!(f, " SET {}", schedule)?; + } + if let Some(num) = suspend_task_after_num_failures { + write!(f, " SUSPEND TASK AFTER {} FAILURES", num)?; + } + if let Some(comments) = comments { + write!(f, " COMMENTS = '{}'", comments)?; + } + Ok(()) + } + AlterTaskOptions::Unset { warehouse } => { + if *warehouse { + write!(f, " UNSET WAREHOUSE")?; + } + Ok(()) + } + AlterTaskOptions::ModifyAs(sql) => write!(f, " AS {}", sql), + } + } +} + +impl Display for AlterTaskStmt { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "ALTER TASK")?; + if self.if_exists { + write!(f, " IF EXISTS")?; + } + write!(f, " {}", self.name)?; + write!(f, "{}", self.options)?; + Ok(()) + } +} + +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct DropTaskStmt { + pub if_exists: bool, + pub name: String, +} + +impl Display for DropTaskStmt { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "DROP TASK")?; + if self.if_exists { + write!(f, " IF EXISTS")?; + } + write!(f, " {}", self.name) + } +} + +#[derive(Debug, Clone, PartialEq)] +pub struct ShowTasksStmt { + pub limit: Option, +} + +impl Display for ShowTasksStmt { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "SHOW ")?; + write!(f, "TASKS")?; + if let Some(limit) = &self.limit { + write!(f, " {limit}")?; + } + + Ok(()) + } +} + +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct ExecuteTaskStmt { + pub name: String, +} + +impl Display for ExecuteTaskStmt { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "EXECUTE TASK {}", self.name) + } +} + +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct DescribeTaskStmt { + pub name: String, +} + +impl Display for DescribeTaskStmt { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "DESCRIBE TASK {}", self.name) + } +} diff --git a/src/query/ast/src/parser/statement.rs b/src/query/ast/src/parser/statement.rs index 169bf5b5b72b..28d6ce025d41 100644 --- a/src/query/ast/src/parser/statement.rs +++ b/src/query/ast/src/parser/statement.rs @@ -135,6 +135,61 @@ pub fn statement(i: Input) -> IResult { }, ); + let alter_task = map( + rule! { + ALTER ~ TASK ~ ( IF ~ ^EXISTS )? + ~ #ident ~ #alter_task_option + }, + |(_, _, opt_if_exists, task, options)| { + Statement::AlterTask(AlterTaskStmt { + if_exists: opt_if_exists.is_some(), + name: task.to_string(), + options, + }) + }, + ); + + let drop_task = map( + rule! { + DROP ~ TASK ~ ( IF ~ ^EXISTS )? + ~ #ident + }, + |(_, _, opt_if_exists, task)| { + Statement::DropTask(DropTaskStmt { + if_exists: opt_if_exists.is_some(), + name: task.to_string(), + }) + }, + ); + let show_tasks = map( + rule! { + SHOW ~ TASKS ~ #show_limit? + }, + |(_, _, limit)| Statement::ShowTasks(ShowTasksStmt { limit }), + ); + + let execute_task = map( + rule! { + EXECUTE ~ TASK ~ #ident + }, + |(_, _, task)| { + Statement::ExecuteTask(ExecuteTaskStmt { + name: task.to_string(), + }) + }, + ); + + let desc_task = map( + rule! { + ( DESC | DESCRIBE ) ~ TASK ~ #ident + }, + |(_, _, task)| { + Statement::DescribeTask(DescribeTaskStmt { + name: task.to_string(), + }) + }, + ); + let insert = map( rule! { INSERT ~ #hint? ~ ( INTO | OVERWRITE ) ~ TABLE? @@ -1594,6 +1649,11 @@ pub fn statement(i: Input) -> IResult { [ COMMENT = '' ] AS `" + | #drop_task : "`DROP TASK [ IF EXISTS ] `" + | #alter_task : "`ALTER TASK [ IF EXISTS ] SUSPEND | RESUME | SET