From e0cfb82d96c281a4b5fa4114fc8064fae093c650 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Tue, 5 Dec 2023 12:25:38 +0800 Subject: [PATCH 001/215] feat(expr): add json path functions (#13568) Signed-off-by: Runji Wang --- Cargo.lock | 18 +- proto/expr.proto | 5 + src/common/src/types/jsonb.rs | 11 +- src/expr/impl/Cargo.toml | 1 + src/expr/impl/src/scalar/jsonb_path.rs | 324 ++++++++++++ src/expr/impl/src/scalar/mod.rs | 1 + src/expr/macro/src/gen.rs | 11 +- src/frontend/src/binder/expr/binary_op.rs | 22 +- src/frontend/src/binder/expr/function.rs | 4 + src/frontend/src/expr/mod.rs | 8 +- src/frontend/src/expr/pure.rs | 4 + src/sqlparser/src/ast/operator.rs | 4 + src/sqlparser/src/parser.rs | 6 +- src/sqlparser/src/tokenizer.rs | 8 + .../regress/data/expected/jsonb_jsonpath.out | 120 ++--- src/tests/regress/data/schedule | 2 +- src/tests/regress/data/sql/jsonb_jsonpath.sql | 498 +++++++++--------- 17 files changed, 724 insertions(+), 323 deletions(-) create mode 100644 src/expr/impl/src/scalar/jsonb_path.rs diff --git a/Cargo.lock b/Cargo.lock index 0f1bd68ef0c6d..cd582cfba90ce 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4347,9 +4347,9 @@ dependencies = [ [[package]] name = "jsonbb" -version = "0.1.2" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f2dc222074a1d88750778b7b0ed81925cb9f36443df447ab11c51d5686609b2" +checksum = "8e2edfc17ad42a5ece82df036301d5ef0c3dc3d071e28aa8a62e461711c55d19" dependencies = [ "bytes", "serde", @@ -7882,6 +7882,7 @@ dependencies = [ "serde_json", "sha1", "sha2", + "sql-json-path", "thiserror", "tracing", "workspace-hack", @@ -9742,6 +9743,19 @@ dependencies = [ "der 0.7.8", ] +[[package]] +name = "sql-json-path" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06d900e5c1db024a5bd4535916f1779f104562a7357ff4280e8136eb50b04f35" +dependencies = [ + "jsonbb", + "nom", + "regex", + "serde_json", + "thiserror", +] + [[package]] name = "sqlformat" version = "0.2.2" diff --git a/proto/expr.proto b/proto/expr.proto index ad393b1df0c7a..653cbeaa339ad 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -253,6 +253,10 @@ message ExprNode { TO_JSONB = 617; JSONB_BUILD_ARRAY = 618; JSONB_BUILD_OBJECT = 619; + JSONB_PATH_EXISTS = 620; + JSONB_PATH_MATCH = 621; + JSONB_PATH_QUERY_ARRAY = 622; + JSONB_PATH_QUERY_FIRST = 623; // Non-pure functions below (> 1000) // ------------------------ @@ -295,6 +299,7 @@ message TableFunction { JSONB_EACH = 12; JSONB_EACH_TEXT = 13; JSONB_OBJECT_KEYS = 14; + JSONB_PATH_QUERY = 15; // User defined table function UDTF = 100; } diff --git a/src/common/src/types/jsonb.rs b/src/common/src/types/jsonb.rs index 71f33a1d53822..a0a4cb87e6b99 100644 --- a/src/common/src/types/jsonb.rs +++ b/src/common/src/types/jsonb.rs @@ -235,9 +235,14 @@ impl<'a> JsonbRef<'a> { buf } + /// Returns a jsonb `null` value. + pub fn null() -> Self { + Self(ValueRef::Null) + } + /// Returns true if this is a jsonb `null`. pub fn is_jsonb_null(&self) -> bool { - self.0.as_null().is_some() + self.0.is_null() } /// Returns true if this is a jsonb null, boolean, number or string. @@ -250,12 +255,12 @@ impl<'a> JsonbRef<'a> { /// Returns true if this is a jsonb array. pub fn is_array(&self) -> bool { - matches!(self.0, ValueRef::Array(_)) + self.0.is_array() } /// Returns true if this is a jsonb object. pub fn is_object(&self) -> bool { - matches!(self.0, ValueRef::Object(_)) + self.0.is_object() } /// Returns the type name of this jsonb. diff --git a/src/expr/impl/Cargo.toml b/src/expr/impl/Cargo.toml index ee6d52977cf6d..309a570dd253b 100644 --- a/src/expr/impl/Cargo.toml +++ b/src/expr/impl/Cargo.toml @@ -42,6 +42,7 @@ serde = { version = "1", features = ["derive"] } serde_json = "1" sha1 = "0.10" sha2 = "0.10" +sql-json-path = { version = "0.1", features = ["jsonbb"] } thiserror = "1" tokio = { version = "0.2", package = "madsim-tokio", features = ["time"] } tracing = "0.1" diff --git a/src/expr/impl/src/scalar/jsonb_path.rs b/src/expr/impl/src/scalar/jsonb_path.rs new file mode 100644 index 0000000000000..85fdf7c6b67e6 --- /dev/null +++ b/src/expr/impl/src/scalar/jsonb_path.rs @@ -0,0 +1,324 @@ +// Copyright 2023 RisingWave 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. + +use jsonbb::ValueRef; +use risingwave_common::types::{JsonbRef, JsonbVal}; +use risingwave_expr::{function, ExprError, Result}; +use sql_json_path::{EvalError, JsonPath, ParseError}; + +#[function( + "jsonb_path_exists(jsonb, varchar) -> boolean", + prebuild = "JsonPath::new($1).map_err(parse_error)?" +)] +fn jsonb_path_exists2(target: JsonbRef<'_>, path: &JsonPath) -> Result { + path.exists::>(target.into()) + .map_err(eval_error) +} + +#[function( + "jsonb_path_exists(jsonb, varchar, jsonb) -> boolean", + prebuild = "JsonPath::new($1).map_err(parse_error)?" +)] +fn jsonb_path_exists3(target: JsonbRef<'_>, vars: JsonbRef<'_>, path: &JsonPath) -> Result { + path.exists_with_vars::>(target.into(), vars.into()) + .map_err(eval_error) +} + +/// Checks whether the JSON path returns any item for the specified JSON value. +/// If the vars argument is specified, it must be a JSON object, and its fields +/// provide named values to be substituted into the jsonpath expression. If the +/// silent argument is specified and is true, the function suppresses the same +/// errors as the @? and @@ operators do. +/// +/// # Examples +/// +/// ```slt +/// query B +/// select jsonb_path_exists('{"a":[1,2,3,4,5]}', '$.a[*] ? (@ >= $min && @ <= $max)', '{"min":2, "max":4}'); +/// ---- +/// t +/// ``` +#[function( + "jsonb_path_exists(jsonb, varchar, jsonb, boolean) -> boolean", + prebuild = "JsonPath::new($1).map_err(parse_error)?" +)] +fn jsonb_path_exists4( + target: JsonbRef<'_>, + vars: JsonbRef<'_>, + silent: bool, + path: &JsonPath, +) -> Result> { + match jsonb_path_exists3(target, vars, path) { + Ok(x) => Ok(Some(x)), + Err(_) if silent => Ok(None), + Err(e) => Err(e), + } +} + +#[function( + "jsonb_path_match(jsonb, varchar) -> boolean", + prebuild = "JsonPath::new($1).map_err(parse_error)?" +)] +fn jsonb_path_match2(target: JsonbRef<'_>, path: &JsonPath) -> Result> { + let matched = path + .query::>(target.into()) + .map_err(eval_error)?; + + if matched.len() != 1 || !matched[0].as_ref().is_boolean() && !matched[0].as_ref().is_null() { + return Err(ExprError::InvalidParam { + name: "jsonb_path_match", + reason: "single boolean result is expected".into(), + }); + } + Ok(matched[0].as_ref().as_bool()) +} + +#[function( + "jsonb_path_match(jsonb, varchar, jsonb) -> boolean", + prebuild = "JsonPath::new($1).map_err(parse_error)?" +)] +fn jsonb_path_match3( + target: JsonbRef<'_>, + vars: JsonbRef<'_>, + path: &JsonPath, +) -> Result> { + let matched = path + .query_with_vars::>(target.into(), vars.into()) + .map_err(eval_error)?; + + if matched.len() != 1 || !matched[0].as_ref().is_boolean() && !matched[0].as_ref().is_null() { + return Err(ExprError::InvalidParam { + name: "jsonb_path_match", + reason: "single boolean result is expected".into(), + }); + } + Ok(matched[0].as_ref().as_bool()) +} + +/// Returns the result of a JSON path predicate check for the specified JSON value. +/// Only the first item of the result is taken into account. +/// If the result is not Boolean, then NULL is returned. +/// The optional vars and silent arguments act the same as for `jsonb_path_exists`. +/// +/// # Examples +/// +/// ```slt +/// query B +/// select jsonb_path_match('{"a":[1,2,3,4,5]}', 'exists($.a[*] ? (@ >= $min && @ <= $max))', '{"min":2, "max":4}'); +/// ---- +/// t +/// ``` +#[function( + "jsonb_path_match(jsonb, varchar, jsonb, boolean) -> boolean", + prebuild = "JsonPath::new($1).map_err(parse_error)?" +)] +fn jsonb_path_match4( + target: JsonbRef<'_>, + vars: JsonbRef<'_>, + silent: bool, + path: &JsonPath, +) -> Result> { + match jsonb_path_match3(target, vars, path) { + Ok(x) => Ok(x), + Err(_) if silent => Ok(None), + Err(e) => Err(e), + } +} + +#[function( + "jsonb_path_query(jsonb, varchar) -> setof jsonb", + prebuild = "JsonPath::new($1).map_err(parse_error)?" +)] +fn jsonb_path_query2<'a>( + target: JsonbRef<'a>, + path: &JsonPath, +) -> Result + 'a> { + let matched = path + .query::>(target.into()) + .map_err(eval_error)?; + Ok(matched.into_iter().map(|json| json.into_owned().into())) +} + +#[function( + "jsonb_path_query(jsonb, varchar, jsonb) -> setof jsonb", + prebuild = "JsonPath::new($1).map_err(parse_error)?" +)] +fn jsonb_path_query3<'a>( + target: JsonbRef<'a>, + vars: JsonbRef<'a>, + path: &JsonPath, +) -> Result + 'a> { + let matched = path + .query_with_vars::>(target.into(), vars.into()) + .map_err(eval_error)?; + Ok(matched.into_iter().map(|json| json.into_owned().into())) +} + +/// Returns all JSON items returned by the JSON path for the specified JSON value. +/// The optional vars and silent arguments act the same as for `jsonb_path_exists`. +/// +/// # Examples +/// +/// ```slt +/// query I +/// select * from jsonb_path_query(jsonb '{"a":[1,2,3,4,5]}', '$.a[*] ? (@ >= $min && @ <= $max)', jsonb '{"min":2, "max":4}'); +/// ---- +/// 2 +/// 3 +/// 4 +/// ``` +#[function( + "jsonb_path_query(jsonb, varchar, jsonb, boolean) -> setof jsonb", + prebuild = "JsonPath::new($1).map_err(parse_error)?" +)] +fn jsonb_path_query4<'a>( + target: JsonbRef<'a>, + vars: JsonbRef<'a>, + silent: bool, + path: &JsonPath, +) -> Result + 'a>> { + match jsonb_path_query3(target, vars, path) { + Ok(x) => Ok(Some(x)), + Err(_) if silent => Ok(None), + Err(e) => Err(e), + } +} + +#[function( + "jsonb_path_query_array(jsonb, varchar) -> jsonb", + prebuild = "JsonPath::new($1).map_err(parse_error)?" +)] +fn jsonb_path_query_array2(target: JsonbRef<'_>, path: &JsonPath) -> Result { + let matched = path + .query::>(target.into()) + .map_err(eval_error)?; + let array = jsonbb::Value::array(matched.iter().map(|json| json.as_ref())); + Ok(array.into()) +} + +#[function( + "jsonb_path_query_array(jsonb, varchar, jsonb) -> jsonb", + prebuild = "JsonPath::new($1).map_err(parse_error)?" +)] +fn jsonb_path_query_array3( + target: JsonbRef<'_>, + vars: JsonbRef<'_>, + path: &JsonPath, +) -> Result { + let matched = path + .query_with_vars::>(target.into(), vars.into()) + .map_err(eval_error)?; + let array = jsonbb::Value::array(matched.iter().map(|json| json.as_ref())); + Ok(array.into()) +} + +/// Returns all JSON items returned by the JSON path for the specified JSON value, as a JSON array. +/// The optional vars and silent arguments act the same as for `jsonb_path_exists`. +/// +/// # Examples +/// +/// ```slt +/// query T +/// select jsonb_path_query_array('{"a":[1,2,3,4,5]}', '$.a[*] ? (@ >= $min && @ <= $max)', '{"min":2, "max":4}'); +/// ---- +/// [2, 3, 4] +/// ``` +#[function( + "jsonb_path_query_array(jsonb, varchar, jsonb, boolean) -> jsonb", + prebuild = "JsonPath::new($1).map_err(parse_error)?" +)] +fn jsonb_path_query_array4( + target: JsonbRef<'_>, + vars: JsonbRef<'_>, + silent: bool, + path: &JsonPath, +) -> Result> { + match jsonb_path_query_array3(target, vars, path) { + Ok(x) => Ok(Some(x)), + Err(_) if silent => Ok(None), + Err(e) => Err(e), + } +} + +#[function( + "jsonb_path_query_first(jsonb, varchar) -> jsonb", + prebuild = "JsonPath::new($1).map_err(parse_error)?" +)] +fn jsonb_path_query_first2(target: JsonbRef<'_>, path: &JsonPath) -> Result> { + let matched = path + .query_first::>(target.into()) + .map_err(eval_error)?; + Ok(matched + .into_iter() + .next() + .map(|json| json.into_owned().into())) +} + +#[function( + "jsonb_path_query_first(jsonb, varchar, jsonb) -> jsonb", + prebuild = "JsonPath::new($1).map_err(parse_error)?" +)] +fn jsonb_path_query_first3( + target: JsonbRef<'_>, + vars: JsonbRef<'_>, + path: &JsonPath, +) -> Result> { + let matched = path + .query_first_with_vars::>(target.into(), vars.into()) + .map_err(eval_error)?; + Ok(matched + .into_iter() + .next() + .map(|json| json.into_owned().into())) +} + +/// Returns the first JSON item returned by the JSON path for the specified JSON value. +/// Returns NULL if there are no results. +/// The optional vars and silent arguments act the same as for `jsonb_path_exists`. +/// +/// # Examples +/// +/// ```slt +/// query T +/// select jsonb_path_query_first('{"a":[1,2,3,4,5]}', '$.a[*] ? (@ >= $min && @ <= $max)', '{"min":2, "max":4}'); +/// ---- +/// 2 +/// ``` +#[function( + "jsonb_path_query_first(jsonb, varchar, jsonb, boolean) -> jsonb", + prebuild = "JsonPath::new($1).map_err(parse_error)?" +)] +fn jsonb_path_query_first4( + target: JsonbRef<'_>, + vars: JsonbRef<'_>, + silent: bool, + path: &JsonPath, +) -> Result> { + match jsonb_path_query_first3(target, vars, path) { + Ok(x) => Ok(x), + Err(_) if silent => Ok(None), + Err(e) => Err(e), + } +} + +fn parse_error(e: ParseError) -> ExprError { + ExprError::Parse(e.to_string().into()) +} + +fn eval_error(e: EvalError) -> ExprError { + ExprError::InvalidParam { + name: "jsonpath", + reason: e.to_string().into(), + } +} diff --git a/src/expr/impl/src/scalar/mod.rs b/src/expr/impl/src/scalar/mod.rs index e99a71323b617..21620d5a6a4e2 100644 --- a/src/expr/impl/src/scalar/mod.rs +++ b/src/expr/impl/src/scalar/mod.rs @@ -50,6 +50,7 @@ mod jsonb_contains; mod jsonb_delete; mod jsonb_info; mod jsonb_object; +mod jsonb_path; mod length; mod lower; mod make_timestamptz; diff --git a/src/expr/macro/src/gen.rs b/src/expr/macro/src/gen.rs index ba01bf05d6f27..2ecac59994e8d 100644 --- a/src/expr/macro/src/gen.rs +++ b/src/expr/macro/src/gen.rs @@ -945,10 +945,9 @@ impl FunctionAttr { let child: Vec<_> = arg_ids.iter().map(|i| format_ident!("child{i}")).collect(); let array_refs: Vec<_> = arg_ids.iter().map(|i| format_ident!("array{i}")).collect(); let arrays: Vec<_> = arg_ids.iter().map(|i| format_ident!("a{i}")).collect(); - let arg_arrays = self - .args + let arg_arrays = arg_ids .iter() - .map(|t| format_ident!("{}", types::array_type(t))); + .map(|i| format_ident!("{}", types::array_type(&self.args[*i]))); let outputs = (0..return_types.len()) .map(|i| format_ident!("o{i}")) .collect_vec(); @@ -1006,9 +1005,11 @@ impl FunctionAttr { }; let iter = match user_fn.return_type_kind { ReturnTypeKind::T => quote! { iter }, - ReturnTypeKind::Option => quote! { iter.flatten() }, + ReturnTypeKind::Option => quote! { if let Some(it) = iter { it } else { continue; } }, ReturnTypeKind::Result => quote! { iter? }, - ReturnTypeKind::ResultOption => quote! { value?.flatten() }, + ReturnTypeKind::ResultOption => { + quote! { if let Some(it) = iter? { it } else { continue; } } + } }; let iterator_item_type = user_fn.iterator_item_kind.clone().ok_or_else(|| { Error::new( diff --git a/src/frontend/src/binder/expr/binary_op.rs b/src/frontend/src/binder/expr/binary_op.rs index 14b55db35ed70..00b8a60d09d60 100644 --- a/src/frontend/src/binder/expr/binary_op.rs +++ b/src/frontend/src/binder/expr/binary_op.rs @@ -14,7 +14,7 @@ use risingwave_common::bail_not_implemented; use risingwave_common::error::{ErrorCode, Result}; -use risingwave_common::types::DataType; +use risingwave_common::types::{DataType, JsonbVal}; use risingwave_sqlparser::ast::{BinaryOperator, Expr}; use crate::binder::Binder; @@ -45,6 +45,26 @@ impl Binder { let bound_right = self.bind_expr_inner(right)?; + if matches!(op, BinaryOperator::PathMatch | BinaryOperator::PathExists) { + // jsonb @? jsonpath => jsonb_path_exists(jsonb, jsonpath, '{}', silent => true) + // jsonb @@ jsonpath => jsonb_path_match(jsonb, jsonpath, '{}', silent => true) + return Ok(FunctionCall::new_unchecked( + match op { + BinaryOperator::PathMatch => ExprType::JsonbPathMatch, + BinaryOperator::PathExists => ExprType::JsonbPathExists, + _ => unreachable!(), + }, + vec![ + bound_left, + bound_right, + ExprImpl::literal_jsonb(JsonbVal::empty_object()), // vars + ExprImpl::literal_bool(true), // silent + ], + DataType::Boolean, + ) + .into()); + } + func_types.extend(Self::resolve_binary_operator( op, &bound_left, diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index a1ff898ba9101..d355c9c3d242e 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -902,6 +902,10 @@ impl Binder { ("to_jsonb", raw_call(ExprType::ToJsonb)), ("jsonb_build_array", raw_call(ExprType::JsonbBuildArray)), ("jsonb_build_object", raw_call(ExprType::JsonbBuildObject)), + ("jsonb_path_match", raw_call(ExprType::JsonbPathMatch)), + ("jsonb_path_exists", raw_call(ExprType::JsonbPathExists)), + ("jsonb_path_query_array", raw_call(ExprType::JsonbPathQueryArray)), + ("jsonb_path_query_first", raw_call(ExprType::JsonbPathQueryFirst)), // Functions that return a constant value ("pi", pi()), // greatest and least diff --git a/src/frontend/src/expr/mod.rs b/src/frontend/src/expr/mod.rs index 2e8b87d89ee88..ecf256fe78de6 100644 --- a/src/frontend/src/expr/mod.rs +++ b/src/frontend/src/expr/mod.rs @@ -18,7 +18,7 @@ use futures::FutureExt; use paste::paste; use risingwave_common::array::ListValue; use risingwave_common::error::{ErrorCode, Result as RwResult}; -use risingwave_common::types::{DataType, Datum, Scalar}; +use risingwave_common::types::{DataType, Datum, JsonbVal, Scalar}; use risingwave_expr::aggregate::AggKind; use risingwave_expr::expr::build_from_prost; use risingwave_pb::expr::expr_node::RexNode; @@ -165,6 +165,12 @@ impl ExprImpl { Literal::new(None, element_type).into() } + /// A literal jsonb value. + #[inline(always)] + pub fn literal_jsonb(v: JsonbVal) -> Self { + Literal::new(Some(v.into()), DataType::Jsonb).into() + } + /// A literal list value. #[inline(always)] pub fn literal_list(v: ListValue, element_type: DataType) -> Self { diff --git a/src/frontend/src/expr/pure.rs b/src/frontend/src/expr/pure.rs index babf9a9df5d29..79ece88731608 100644 --- a/src/frontend/src/expr/pure.rs +++ b/src/frontend/src/expr/pure.rs @@ -190,6 +190,10 @@ impl ExprVisitor for ImpureAnalyzer { | expr_node::Type::JsonbStripNulls | expr_node::Type::JsonbBuildArray | expr_node::Type::JsonbBuildObject + | expr_node::Type::JsonbPathExists + | expr_node::Type::JsonbPathMatch + | expr_node::Type::JsonbPathQueryArray + | expr_node::Type::JsonbPathQueryFirst | expr_node::Type::IsJson | expr_node::Type::ToJsonb | expr_node::Type::Sind diff --git a/src/sqlparser/src/ast/operator.rs b/src/sqlparser/src/ast/operator.rs index 9929d56b28679..d4fca378d7b5b 100644 --- a/src/sqlparser/src/ast/operator.rs +++ b/src/sqlparser/src/ast/operator.rs @@ -105,6 +105,8 @@ pub enum BinaryOperator { Exists, ExistsAny, ExistsAll, + PathMatch, + PathExists, PGQualified(Box), } @@ -155,6 +157,8 @@ impl fmt::Display for BinaryOperator { BinaryOperator::Exists => "?", BinaryOperator::ExistsAny => "?|", BinaryOperator::ExistsAll => "?&", + BinaryOperator::PathMatch => "@@", + BinaryOperator::PathExists => "@?", BinaryOperator::PGQualified(_) => unreachable!(), }) } diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index c8f36b7873b98..fcd3a3dff3e36 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -1405,6 +1405,8 @@ impl Parser { Token::QuestionMark => Some(BinaryOperator::Exists), Token::QuestionMarkPipe => Some(BinaryOperator::ExistsAny), Token::QuestionMarkAmpersand => Some(BinaryOperator::ExistsAll), + Token::AtQuestionMark => Some(BinaryOperator::PathExists), + Token::AtAt => Some(BinaryOperator::PathMatch), Token::Word(w) => match w.keyword { Keyword::AND => Some(BinaryOperator::And), Keyword::OR => Some(BinaryOperator::Or), @@ -1749,7 +1751,9 @@ impl Parser { | Token::ArrowAt | Token::QuestionMark | Token::QuestionMarkPipe - | Token::QuestionMarkAmpersand => Ok(P::Other), + | Token::QuestionMarkAmpersand + | Token::AtQuestionMark + | Token::AtAt => Ok(P::Other), Token::Word(w) if w.keyword == Keyword::OPERATOR && self.peek_nth_token(1) == Token::LParen => { diff --git a/src/sqlparser/src/tokenizer.rs b/src/sqlparser/src/tokenizer.rs index e33260e3e2213..2a36f5ba7e8f0 100644 --- a/src/sqlparser/src/tokenizer.rs +++ b/src/sqlparser/src/tokenizer.rs @@ -176,6 +176,10 @@ pub enum Token { QuestionMarkPipe, /// `?&`, do all of the strings exist as top-level keys or array elements? QuestionMarkAmpersand, + /// `@?`, does JSON path return any item for the specified JSON value? + AtQuestionMark, + /// `@@`, returns the result of a JSON path predicate check for the specified JSON value. + AtAt, } impl fmt::Display for Token { @@ -249,6 +253,8 @@ impl fmt::Display for Token { Token::QuestionMark => f.write_str("?"), Token::QuestionMarkPipe => f.write_str("?|"), Token::QuestionMarkAmpersand => f.write_str("?&"), + Token::AtQuestionMark => f.write_str("@?"), + Token::AtAt => f.write_str("@@"), } } } @@ -780,6 +786,8 @@ impl<'a> Tokenizer<'a> { chars.next(); // consume the '@' match chars.peek() { Some('>') => self.consume_and_return(chars, Token::AtArrow), + Some('?') => self.consume_and_return(chars, Token::AtQuestionMark), + Some('@') => self.consume_and_return(chars, Token::AtAt), // a regular '@' operator _ => Ok(Some(Token::AtSign)), } diff --git a/src/tests/regress/data/expected/jsonb_jsonpath.out b/src/tests/regress/data/expected/jsonb_jsonpath.out index 508ddd797ed59..c61817fc8070f 100644 --- a/src/tests/regress/data/expected/jsonb_jsonpath.out +++ b/src/tests/regress/data/expected/jsonb_jsonpath.out @@ -120,7 +120,7 @@ select jsonb '[1]' @? 'strict $[1]'; select jsonb_path_query('[1]', 'strict $[1]'); ERROR: jsonpath array subscript is out of bounds -select jsonb_path_query('[1]', 'strict $[1]', silent => true); +select jsonb_path_query('[1]', 'strict $[1]', '{}', true); jsonb_path_query ------------------ (0 rows) @@ -225,21 +225,21 @@ select jsonb '[{"a": 1}, {"a": 2}]' @? '$[0 to 1] ? (@.a > 1)'; t (1 row) -select jsonb_path_exists('[{"a": 1}, {"a": 2}, 3]', 'lax $[*].a', silent => false); +select jsonb_path_exists('[{"a": 1}, {"a": 2}, 3]', 'lax $[*].a', '{}', false); jsonb_path_exists ------------------- t (1 row) -select jsonb_path_exists('[{"a": 1}, {"a": 2}, 3]', 'lax $[*].a', silent => true); +select jsonb_path_exists('[{"a": 1}, {"a": 2}, 3]', 'lax $[*].a', '{}', true); jsonb_path_exists ------------------- t (1 row) -select jsonb_path_exists('[{"a": 1}, {"a": 2}, 3]', 'strict $[*].a', silent => false); +select jsonb_path_exists('[{"a": 1}, {"a": 2}, 3]', 'strict $[*].a', '{}', false); ERROR: jsonpath member accessor can only be applied to an object -select jsonb_path_exists('[{"a": 1}, {"a": 2}, 3]', 'strict $[*].a', silent => true); +select jsonb_path_exists('[{"a": 1}, {"a": 2}, 3]', 'strict $[*].a', '{}', true); jsonb_path_exists ------------------- @@ -254,12 +254,12 @@ select jsonb_path_query('1', 'strict $.a'); ERROR: jsonpath member accessor can only be applied to an object select jsonb_path_query('1', 'strict $.*'); ERROR: jsonpath wildcard member accessor can only be applied to an object -select jsonb_path_query('1', 'strict $.a', silent => true); +select jsonb_path_query('1', 'strict $.a', '{}', true); jsonb_path_query ------------------ (0 rows) -select jsonb_path_query('1', 'strict $.*', silent => true); +select jsonb_path_query('1', 'strict $.*', '{}', true); jsonb_path_query ------------------ (0 rows) @@ -271,7 +271,7 @@ select jsonb_path_query('[]', 'lax $.a'); select jsonb_path_query('[]', 'strict $.a'); ERROR: jsonpath member accessor can only be applied to an object -select jsonb_path_query('[]', 'strict $.a', silent => true); +select jsonb_path_query('[]', 'strict $.a', '{}', true); jsonb_path_query ------------------ (0 rows) @@ -283,7 +283,7 @@ select jsonb_path_query('{}', 'lax $.a'); select jsonb_path_query('{}', 'strict $.a'); ERROR: JSON object does not contain key "a" -select jsonb_path_query('{}', 'strict $.a', silent => true); +select jsonb_path_query('{}', 'strict $.a', '{}', true); jsonb_path_query ------------------ (0 rows) @@ -296,22 +296,22 @@ select jsonb_path_query('[]', 'strict $[1]'); ERROR: jsonpath array subscript is out of bounds select jsonb_path_query('[]', 'strict $["a"]'); ERROR: jsonpath array subscript is not a single numeric value -select jsonb_path_query('1', 'strict $[1]', silent => true); +select jsonb_path_query('1', 'strict $[1]', '{}', true); jsonb_path_query ------------------ (0 rows) -select jsonb_path_query('1', 'strict $[*]', silent => true); +select jsonb_path_query('1', 'strict $[*]', '{}', true); jsonb_path_query ------------------ (0 rows) -select jsonb_path_query('[]', 'strict $[1]', silent => true); +select jsonb_path_query('[]', 'strict $[1]', '{}', true); jsonb_path_query ------------------ (0 rows) -select jsonb_path_query('[]', 'strict $["a"]', silent => true); +select jsonb_path_query('[]', 'strict $["a"]', '{}', true); jsonb_path_query ------------------ (0 rows) @@ -426,7 +426,7 @@ select jsonb_path_query('[1,2,3]', 'lax $[*]'); select jsonb_path_query('[1,2,3]', 'strict $[*].a'); ERROR: jsonpath member accessor can only be applied to an object -select jsonb_path_query('[1,2,3]', 'strict $[*].a', silent => true); +select jsonb_path_query('[1,2,3]', 'strict $[*].a', '{}', true); jsonb_path_query ------------------ (0 rows) @@ -443,7 +443,7 @@ select jsonb_path_query('[]', '$[last ? (exists(last))]'); select jsonb_path_query('[]', 'strict $[last]'); ERROR: jsonpath array subscript is out of bounds -select jsonb_path_query('[]', 'strict $[last]', silent => true); +select jsonb_path_query('[]', 'strict $[last]', '{}', true); jsonb_path_query ------------------ (0 rows) @@ -474,7 +474,7 @@ select jsonb_path_query('[1,2,3]', '$[last ? (@.type() == "number")]'); select jsonb_path_query('[1,2,3]', '$[last ? (@.type() == "string")]'); ERROR: jsonpath array subscript is not a single numeric value -select jsonb_path_query('[1,2,3]', '$[last ? (@.type() == "string")]', silent => true); +select jsonb_path_query('[1,2,3]', '$[last ? (@.type() == "string")]', '{}', true); jsonb_path_query ------------------ (0 rows) @@ -1061,22 +1061,22 @@ select jsonb_path_query('"a"', '-$'); ERROR: operand of unary jsonpath operator - is not a numeric value select jsonb_path_query('[1,"2",3]', '+$'); ERROR: operand of unary jsonpath operator + is not a numeric value -select jsonb_path_query('1', '$ + "2"', silent => true); +select jsonb_path_query('1', '$ + "2"', '{}', true); jsonb_path_query ------------------ (0 rows) -select jsonb_path_query('[1, 2]', '3 * $', silent => true); +select jsonb_path_query('[1, 2]', '3 * $', '{}', true); jsonb_path_query ------------------ (0 rows) -select jsonb_path_query('"a"', '-$', silent => true); +select jsonb_path_query('"a"', '-$', '{}', true); jsonb_path_query ------------------ (0 rows) -select jsonb_path_query('[1,"2",3]', '+$', silent => true); +select jsonb_path_query('[1,"2",3]', '+$', '{}', true); jsonb_path_query ------------------ 1 @@ -1130,7 +1130,7 @@ select jsonb_path_query('{"a": [2, 3, 4]}', 'lax -$.a'); -- should fail select jsonb_path_query('{"a": [1, 2]}', 'lax $.a * 3'); ERROR: left operand of jsonpath operator * is not a single numeric value -select jsonb_path_query('{"a": [1, 2]}', 'lax $.a * 3', silent => true); +select jsonb_path_query('{"a": [1, 2]}', 'lax $.a * 3', '{}', true); jsonb_path_query ------------------ (0 rows) @@ -1220,25 +1220,25 @@ select jsonb_path_match('[[1, true], [2, false]]', 'strict $[*] ? (@[0] < $x) [1 t (1 row) -select jsonb_path_match('[{"a": 1}, {"a": 2}, 3]', 'lax exists($[*].a)', silent => false); +select jsonb_path_match('[{"a": 1}, {"a": 2}, 3]', 'lax exists($[*].a)', '{}', false); jsonb_path_match ------------------ t (1 row) -select jsonb_path_match('[{"a": 1}, {"a": 2}, 3]', 'lax exists($[*].a)', silent => true); +select jsonb_path_match('[{"a": 1}, {"a": 2}, 3]', 'lax exists($[*].a)', '{}', true); jsonb_path_match ------------------ t (1 row) -select jsonb_path_match('[{"a": 1}, {"a": 2}, 3]', 'strict exists($[*].a)', silent => false); +select jsonb_path_match('[{"a": 1}, {"a": 2}, 3]', 'strict exists($[*].a)', '{}', false); jsonb_path_match ------------------ (1 row) -select jsonb_path_match('[{"a": 1}, {"a": 2}, 3]', 'strict exists($[*].a)', silent => true); +select jsonb_path_match('[{"a": 1}, {"a": 2}, 3]', 'strict exists($[*].a)', '{}', true); jsonb_path_match ------------------ @@ -1329,7 +1329,7 @@ select jsonb_path_query('[1, 2, 3]', 'strict ($[*].a > 3).type()'); select jsonb_path_query('[1,null,true,"11",[],[1],[1,2,3],{},{"a":1,"b":2}]', 'strict $[*].size()'); ERROR: jsonpath item method .size() can only be applied to an array -select jsonb_path_query('[1,null,true,"11",[],[1],[1,2,3],{},{"a":1,"b":2}]', 'strict $[*].size()', silent => true); +select jsonb_path_query('[1,null,true,"11",[],[1],[1,2,3],{},{"a":1,"b":2}]', 'strict $[*].size()', '{}', true); jsonb_path_query ------------------ (0 rows) @@ -1400,7 +1400,7 @@ select jsonb_path_query('[0, 1, -2, -3.4, 5.6]', '$[*].ceiling().abs().type()'); select jsonb_path_query('[{},1]', '$[*].keyvalue()'); ERROR: jsonpath item method .keyvalue() can only be applied to an object -select jsonb_path_query('[{},1]', '$[*].keyvalue()', silent => true); +select jsonb_path_query('[{},1]', '$[*].keyvalue()', '{}', true); jsonb_path_query ------------------ (0 rows) @@ -1454,12 +1454,12 @@ select jsonb_path_query('null', '$.double()'); ERROR: jsonpath item method .double() can only be applied to a string or numeric value select jsonb_path_query('true', '$.double()'); ERROR: jsonpath item method .double() can only be applied to a string or numeric value -select jsonb_path_query('null', '$.double()', silent => true); +select jsonb_path_query('null', '$.double()', '{}', true); jsonb_path_query ------------------ (0 rows) -select jsonb_path_query('true', '$.double()', silent => true); +select jsonb_path_query('true', '$.double()', '{}', true); jsonb_path_query ------------------ (0 rows) @@ -1473,12 +1473,12 @@ select jsonb_path_query('[]', 'strict $.double()'); ERROR: jsonpath item method .double() can only be applied to a string or numeric value select jsonb_path_query('{}', '$.double()'); ERROR: jsonpath item method .double() can only be applied to a string or numeric value -select jsonb_path_query('[]', 'strict $.double()', silent => true); +select jsonb_path_query('[]', 'strict $.double()', '{}', true); jsonb_path_query ------------------ (0 rows) -select jsonb_path_query('{}', '$.double()', silent => true); +select jsonb_path_query('{}', '$.double()', '{}', true); jsonb_path_query ------------------ (0 rows) @@ -1507,12 +1507,12 @@ select jsonb_path_query('"inf"', '$.double()'); ERROR: string argument of jsonpath item method .double() is not a valid representation of a double precision number select jsonb_path_query('"-inf"', '$.double()'); ERROR: string argument of jsonpath item method .double() is not a valid representation of a double precision number -select jsonb_path_query('"inf"', '$.double()', silent => true); +select jsonb_path_query('"inf"', '$.double()', '{}', true); jsonb_path_query ------------------ (0 rows) -select jsonb_path_query('"-inf"', '$.double()', silent => true); +select jsonb_path_query('"-inf"', '$.double()', '{}', true); jsonb_path_query ------------------ (0 rows) @@ -1523,17 +1523,17 @@ select jsonb_path_query('true', '$.floor()'); ERROR: jsonpath item method .floor() can only be applied to a numeric value select jsonb_path_query('"1.2"', '$.ceiling()'); ERROR: jsonpath item method .ceiling() can only be applied to a numeric value -select jsonb_path_query('{}', '$.abs()', silent => true); +select jsonb_path_query('{}', '$.abs()', '{}', true); jsonb_path_query ------------------ (0 rows) -select jsonb_path_query('true', '$.floor()', silent => true); +select jsonb_path_query('true', '$.floor()', '{}', true); jsonb_path_query ------------------ (0 rows) -select jsonb_path_query('"1.2"', '$.ceiling()', silent => true); +select jsonb_path_query('"1.2"', '$.ceiling()', '{}', true); jsonb_path_query ------------------ (0 rows) @@ -2232,13 +2232,13 @@ SELECT jsonb_path_query_array('[{"a": 1}, {"a": 2}]', '$[*].a ? (@ > 10)'); [] (1 row) -SELECT jsonb_path_query_array('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*].a ? (@ > $min && @ < $max)', vars => '{"min": 1, "max": 4}'); +SELECT jsonb_path_query_array('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*].a ? (@ > $min && @ < $max)', '{"min": 1, "max": 4}'); jsonb_path_query_array ------------------------ [2, 3] (1 row) -SELECT jsonb_path_query_array('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*].a ? (@ > $min && @ < $max)', vars => '{"min": 3, "max": 4}'); +SELECT jsonb_path_query_array('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*].a ? (@ > $min && @ < $max)', '{"min": 3, "max": 4}'); jsonb_path_query_array ------------------------ [] @@ -2246,7 +2246,7 @@ SELECT jsonb_path_query_array('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*]. SELECT jsonb_path_query_first('[{"a": 1}, {"a": 2}, {}]', 'strict $[*].a'); ERROR: JSON object does not contain key "a" -SELECT jsonb_path_query_first('[{"a": 1}, {"a": 2}, {}]', 'strict $[*].a', silent => true); +SELECT jsonb_path_query_first('[{"a": 1}, {"a": 2}, {}]', 'strict $[*].a', '{}', true); jsonb_path_query_first ------------------------ 1 @@ -2270,13 +2270,13 @@ SELECT jsonb_path_query_first('[{"a": 1}, {"a": 2}]', '$[*].a ? (@ > 10)'); (1 row) -SELECT jsonb_path_query_first('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*].a ? (@ > $min && @ < $max)', vars => '{"min": 1, "max": 4}'); +SELECT jsonb_path_query_first('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*].a ? (@ > $min && @ < $max)', '{"min": 1, "max": 4}'); jsonb_path_query_first ------------------------ 2 (1 row) -SELECT jsonb_path_query_first('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*].a ? (@ > $min && @ < $max)', vars => '{"min": 3, "max": 4}'); +SELECT jsonb_path_query_first('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*].a ? (@ > $min && @ < $max)', '{"min": 3, "max": 4}'); jsonb_path_query_first ------------------------ @@ -2300,61 +2300,61 @@ SELECT jsonb_path_exists('[{"a": 1}, {"a": 2}]', '$[*].a ? (@ > 1)'); t (1 row) -SELECT jsonb_path_exists('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*] ? (@.a > $min && @.a < $max)', vars => '{"min": 1, "max": 4}'); +SELECT jsonb_path_exists('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*] ? (@.a > $min && @.a < $max)', '{"min": 1, "max": 4}'); jsonb_path_exists ------------------- t (1 row) -SELECT jsonb_path_exists('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*] ? (@.a > $min && @.a < $max)', vars => '{"min": 3, "max": 4}'); +SELECT jsonb_path_exists('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*] ? (@.a > $min && @.a < $max)', '{"min": 3, "max": 4}'); jsonb_path_exists ------------------- f (1 row) -SELECT jsonb_path_match('true', '$', silent => false); +SELECT jsonb_path_match('true', '$', '{}', false); jsonb_path_match ------------------ t (1 row) -SELECT jsonb_path_match('false', '$', silent => false); +SELECT jsonb_path_match('false', '$', '{}', false); jsonb_path_match ------------------ f (1 row) -SELECT jsonb_path_match('null', '$', silent => false); +SELECT jsonb_path_match('null', '$', '{}', false); jsonb_path_match ------------------ (1 row) -SELECT jsonb_path_match('1', '$', silent => true); +SELECT jsonb_path_match('1', '$', '{}', true); jsonb_path_match ------------------ (1 row) -SELECT jsonb_path_match('1', '$', silent => false); +SELECT jsonb_path_match('1', '$', '{}', false); ERROR: single boolean result is expected -SELECT jsonb_path_match('"a"', '$', silent => false); +SELECT jsonb_path_match('"a"', '$', '{}', false); ERROR: single boolean result is expected -SELECT jsonb_path_match('{}', '$', silent => false); +SELECT jsonb_path_match('{}', '$', '{}', false); ERROR: single boolean result is expected -SELECT jsonb_path_match('[true]', '$', silent => false); +SELECT jsonb_path_match('[true]', '$', '{}', false); ERROR: single boolean result is expected -SELECT jsonb_path_match('{}', 'lax $.a', silent => false); +SELECT jsonb_path_match('{}', 'lax $.a', '{}', false); ERROR: single boolean result is expected -SELECT jsonb_path_match('{}', 'strict $.a', silent => false); +SELECT jsonb_path_match('{}', 'strict $.a', '{}', false); ERROR: JSON object does not contain key "a" -SELECT jsonb_path_match('{}', 'strict $.a', silent => true); +SELECT jsonb_path_match('{}', 'strict $.a', '{}', true); jsonb_path_match ------------------ (1 row) -SELECT jsonb_path_match('[true, true]', '$[*]', silent => false); +SELECT jsonb_path_match('[true, true]', '$[*]', '{}', false); ERROR: single boolean result is expected SELECT jsonb '[{"a": 1}, {"a": 2}]' @@ '$[*].a > 1'; ?column? @@ -2382,11 +2382,11 @@ WITH str(j, num) AS ) SELECT s1.j, s2.j, - jsonb_path_query_first(s1.j, '$.s < $s', vars => s2.j) lt, - jsonb_path_query_first(s1.j, '$.s <= $s', vars => s2.j) le, - jsonb_path_query_first(s1.j, '$.s == $s', vars => s2.j) eq, - jsonb_path_query_first(s1.j, '$.s >= $s', vars => s2.j) ge, - jsonb_path_query_first(s1.j, '$.s > $s', vars => s2.j) gt + jsonb_path_query_first(s1.j, '$.s < $s', s2.j) lt, + jsonb_path_query_first(s1.j, '$.s <= $s', s2.j) le, + jsonb_path_query_first(s1.j, '$.s == $s', s2.j) eq, + jsonb_path_query_first(s1.j, '$.s >= $s', s2.j) ge, + jsonb_path_query_first(s1.j, '$.s > $s', s2.j) gt FROM str s1, str s2 ORDER BY s1.num, s2.num; j | j | lt | le | eq | ge | gt diff --git a/src/tests/regress/data/schedule b/src/tests/regress/data/schedule index 6c424e7ba299d..044e6b40516f1 100644 --- a/src/tests/regress/data/schedule +++ b/src/tests/regress/data/schedule @@ -10,5 +10,5 @@ test: boolean varchar text int2 int4 int8 float4 float8 comments test: strings date time timestamp interval test: case arrays delete -test: jsonb +test: jsonb jsonb_jsonpath test: regex diff --git a/src/tests/regress/data/sql/jsonb_jsonpath.sql b/src/tests/regress/data/sql/jsonb_jsonpath.sql index 60f73cb05906c..867722fc3240b 100644 --- a/src/tests/regress/data/sql/jsonb_jsonpath.sql +++ b/src/tests/regress/data/sql/jsonb_jsonpath.sql @@ -19,7 +19,7 @@ select jsonb '[1]' @? '$[*]'; select jsonb '[1]' @? '$[1]'; select jsonb '[1]' @? 'strict $[1]'; select jsonb_path_query('[1]', 'strict $[1]'); -select jsonb_path_query('[1]', 'strict $[1]', silent => true); +select jsonb_path_query('[1]', 'strict $[1]', '{}', true); select jsonb '[1]' @? 'lax $[10000000000000000]'; select jsonb '[1]' @? 'strict $[10000000000000000]'; select jsonb_path_query('[1]', 'lax $[10000000000000000]'); @@ -39,31 +39,31 @@ select jsonb '1' @? '$ ? ((@ == "1") is unknown)'; select jsonb '1' @? '$ ? ((@ == 1) is unknown)'; select jsonb '[{"a": 1}, {"a": 2}]' @? '$[0 to 1] ? (@.a > 1)'; -select jsonb_path_exists('[{"a": 1}, {"a": 2}, 3]', 'lax $[*].a', silent => false); -select jsonb_path_exists('[{"a": 1}, {"a": 2}, 3]', 'lax $[*].a', silent => true); -select jsonb_path_exists('[{"a": 1}, {"a": 2}, 3]', 'strict $[*].a', silent => false); -select jsonb_path_exists('[{"a": 1}, {"a": 2}, 3]', 'strict $[*].a', silent => true); +select jsonb_path_exists('[{"a": 1}, {"a": 2}, 3]', 'lax $[*].a', '{}', false); +select jsonb_path_exists('[{"a": 1}, {"a": 2}, 3]', 'lax $[*].a', '{}', true); +select jsonb_path_exists('[{"a": 1}, {"a": 2}, 3]', 'strict $[*].a', '{}', false); +select jsonb_path_exists('[{"a": 1}, {"a": 2}, 3]', 'strict $[*].a', '{}', true); select jsonb_path_query('1', 'lax $.a'); select jsonb_path_query('1', 'strict $.a'); select jsonb_path_query('1', 'strict $.*'); -select jsonb_path_query('1', 'strict $.a', silent => true); -select jsonb_path_query('1', 'strict $.*', silent => true); +select jsonb_path_query('1', 'strict $.a', '{}', true); +select jsonb_path_query('1', 'strict $.*', '{}', true); select jsonb_path_query('[]', 'lax $.a'); select jsonb_path_query('[]', 'strict $.a'); -select jsonb_path_query('[]', 'strict $.a', silent => true); +select jsonb_path_query('[]', 'strict $.a', '{}', true); select jsonb_path_query('{}', 'lax $.a'); select jsonb_path_query('{}', 'strict $.a'); -select jsonb_path_query('{}', 'strict $.a', silent => true); +select jsonb_path_query('{}', 'strict $.a', '{}', true); select jsonb_path_query('1', 'strict $[1]'); select jsonb_path_query('1', 'strict $[*]'); select jsonb_path_query('[]', 'strict $[1]'); select jsonb_path_query('[]', 'strict $["a"]'); -select jsonb_path_query('1', 'strict $[1]', silent => true); -select jsonb_path_query('1', 'strict $[*]', silent => true); -select jsonb_path_query('[]', 'strict $[1]', silent => true); -select jsonb_path_query('[]', 'strict $["a"]', silent => true); +select jsonb_path_query('1', 'strict $[1]', '{}', true); +select jsonb_path_query('1', 'strict $[*]', '{}', true); +select jsonb_path_query('[]', 'strict $[1]', '{}', true); +select jsonb_path_query('[]', 'strict $["a"]', '{}', true); select jsonb_path_query('{"a": 12, "b": {"a": 13}}', '$.a'); select jsonb_path_query('{"a": 12, "b": {"a": 13}}', '$.b'); @@ -84,17 +84,17 @@ select jsonb_path_query('[1]', 'lax $[0]'); select jsonb_path_query('[1]', 'lax $[*]'); select jsonb_path_query('[1,2,3]', 'lax $[*]'); select jsonb_path_query('[1,2,3]', 'strict $[*].a'); -select jsonb_path_query('[1,2,3]', 'strict $[*].a', silent => true); +select jsonb_path_query('[1,2,3]', 'strict $[*].a', '{}', true); select jsonb_path_query('[]', '$[last]'); select jsonb_path_query('[]', '$[last ? (exists(last))]'); select jsonb_path_query('[]', 'strict $[last]'); -select jsonb_path_query('[]', 'strict $[last]', silent => true); +select jsonb_path_query('[]', 'strict $[last]', '{}', true); select jsonb_path_query('[1]', '$[last]'); select jsonb_path_query('[1,2,3]', '$[last]'); select jsonb_path_query('[1,2,3]', '$[last - 1]'); select jsonb_path_query('[1,2,3]', '$[last ? (@.type() == "number")]'); select jsonb_path_query('[1,2,3]', '$[last ? (@.type() == "string")]'); -select jsonb_path_query('[1,2,3]', '$[last ? (@.type() == "string")]', silent => true); +select jsonb_path_query('[1,2,3]', '$[last ? (@.type() == "string")]', '{}', true); select * from jsonb_path_query('{"a": 10}', '$'); select * from jsonb_path_query('{"a": 10}', '$ ? (@.a < $value)'); @@ -224,11 +224,11 @@ select jsonb_path_query('1', '$ + "2"'); select jsonb_path_query('[1, 2]', '3 * $'); select jsonb_path_query('"a"', '-$'); select jsonb_path_query('[1,"2",3]', '+$'); -select jsonb_path_query('1', '$ + "2"', silent => true); -select jsonb_path_query('[1, 2]', '3 * $', silent => true); -select jsonb_path_query('"a"', '-$', silent => true); -select jsonb_path_query('[1,"2",3]', '+$', silent => true); -select jsonb '["1",2,0,3]' @? '-$[*]'; +select jsonb_path_query('1', '$ + "2"', '{}', true); +select jsonb_path_query('[1, 2]', '3 * $', '{}', true); +select jsonb_path_query('"a"', '-$', '{}', true); +--@ select jsonb_path_query('[1,"2",3]', '+$', '{}', true); +--@ select jsonb '["1",2,0,3]' @? '-$[*]'; select jsonb '[1,"2",0,3]' @? '-$[*]'; select jsonb '["1",2,0,3]' @? 'strict -$[*]'; select jsonb '[1,"2",0,3]' @? 'strict -$[*]'; @@ -239,7 +239,7 @@ select jsonb_path_query('{"a": [2]}', 'lax $.a + 3'); select jsonb_path_query('{"a": [2, 3, 4]}', 'lax -$.a'); -- should fail select jsonb_path_query('{"a": [1, 2]}', 'lax $.a * 3'); -select jsonb_path_query('{"a": [1, 2]}', 'lax $.a * 3', silent => true); +select jsonb_path_query('{"a": [1, 2]}', 'lax $.a * 3', '{}', true); -- extension: boolean expressions select jsonb_path_query('2', '$ > 1'); @@ -258,10 +258,10 @@ select jsonb '[]' @@ '$[*]'; select jsonb_path_match('[[1, true], [2, false]]', 'strict $[*] ? (@[0] > $x) [1]', '{"x": 1}'); select jsonb_path_match('[[1, true], [2, false]]', 'strict $[*] ? (@[0] < $x) [1]', '{"x": 2}'); -select jsonb_path_match('[{"a": 1}, {"a": 2}, 3]', 'lax exists($[*].a)', silent => false); -select jsonb_path_match('[{"a": 1}, {"a": 2}, 3]', 'lax exists($[*].a)', silent => true); -select jsonb_path_match('[{"a": 1}, {"a": 2}, 3]', 'strict exists($[*].a)', silent => false); -select jsonb_path_match('[{"a": 1}, {"a": 2}, 3]', 'strict exists($[*].a)', silent => true); +select jsonb_path_match('[{"a": 1}, {"a": 2}, 3]', 'lax exists($[*].a)', '{}', false); +select jsonb_path_match('[{"a": 1}, {"a": 2}, 3]', 'lax exists($[*].a)', '{}', true); +select jsonb_path_match('[{"a": 1}, {"a": 2}, 3]', 'strict exists($[*].a)', '{}', false); +select jsonb_path_match('[{"a": 1}, {"a": 2}, 3]', 'strict exists($[*].a)', '{}', true); select jsonb_path_query('[null,1,true,"a",[],{}]', '$.type()'); @@ -273,14 +273,14 @@ select jsonb_path_query('null', '(123).type()'); select jsonb_path_query('null', '"123".type()'); select jsonb_path_query('{"a": 2}', '($.a - 5).abs() + 10'); -select jsonb_path_query('{"a": 2.5}', '-($.a * $.a).floor() % 4.3'); +--@ select jsonb_path_query('{"a": 2.5}', '-($.a * $.a).floor() % 4.3'); select jsonb_path_query('[1, 2, 3]', '($[*] > 2) ? (@ == true)'); select jsonb_path_query('[1, 2, 3]', '($[*] > 3).type()'); select jsonb_path_query('[1, 2, 3]', '($[*].a > 3).type()'); select jsonb_path_query('[1, 2, 3]', 'strict ($[*].a > 3).type()'); select jsonb_path_query('[1,null,true,"11",[],[1],[1,2,3],{},{"a":1,"b":2}]', 'strict $[*].size()'); -select jsonb_path_query('[1,null,true,"11",[],[1],[1,2,3],{},{"a":1,"b":2}]', 'strict $[*].size()', silent => true); +select jsonb_path_query('[1,null,true,"11",[],[1],[1,2,3],{},{"a":1,"b":2}]', 'strict $[*].size()', '{}', true); select jsonb_path_query('[1,null,true,"11",[],[1],[1,2,3],{},{"a":1,"b":2}]', 'lax $[*].size()'); select jsonb_path_query('[0, 1, -2, -3.4, 5.6]', '$[*].abs()'); @@ -290,25 +290,25 @@ select jsonb_path_query('[0, 1, -2, -3.4, 5.6]', '$[*].ceiling().abs()'); select jsonb_path_query('[0, 1, -2, -3.4, 5.6]', '$[*].ceiling().abs().type()'); select jsonb_path_query('[{},1]', '$[*].keyvalue()'); -select jsonb_path_query('[{},1]', '$[*].keyvalue()', silent => true); +select jsonb_path_query('[{},1]', '$[*].keyvalue()', '{}', true); select jsonb_path_query('{}', '$.keyvalue()'); select jsonb_path_query('{"a": 1, "b": [1, 2], "c": {"a": "bbb"}}', '$.keyvalue()'); -select jsonb_path_query('[{"a": 1, "b": [1, 2]}, {"c": {"a": "bbb"}}]', '$[*].keyvalue()'); +--@ select jsonb_path_query('[{"a": 1, "b": [1, 2]}, {"c": {"a": "bbb"}}]', '$[*].keyvalue()'); select jsonb_path_query('[{"a": 1, "b": [1, 2]}, {"c": {"a": "bbb"}}]', 'strict $.keyvalue()'); -select jsonb_path_query('[{"a": 1, "b": [1, 2]}, {"c": {"a": "bbb"}}]', 'lax $.keyvalue()'); +--@ select jsonb_path_query('[{"a": 1, "b": [1, 2]}, {"c": {"a": "bbb"}}]', 'lax $.keyvalue()'); select jsonb_path_query('[{"a": 1, "b": [1, 2]}, {"c": {"a": "bbb"}}]', 'strict $.keyvalue().a'); select jsonb '{"a": 1, "b": [1, 2]}' @? 'lax $.keyvalue()'; select jsonb '{"a": 1, "b": [1, 2]}' @? 'lax $.keyvalue().key'; select jsonb_path_query('null', '$.double()'); select jsonb_path_query('true', '$.double()'); -select jsonb_path_query('null', '$.double()', silent => true); -select jsonb_path_query('true', '$.double()', silent => true); +select jsonb_path_query('null', '$.double()', '{}', true); +select jsonb_path_query('true', '$.double()', '{}', true); select jsonb_path_query('[]', '$.double()'); select jsonb_path_query('[]', 'strict $.double()'); select jsonb_path_query('{}', '$.double()'); -select jsonb_path_query('[]', 'strict $.double()', silent => true); -select jsonb_path_query('{}', '$.double()', silent => true); +select jsonb_path_query('[]', 'strict $.double()', '{}', true); +select jsonb_path_query('{}', '$.double()', '{}', true); select jsonb_path_query('1.23', '$.double()'); select jsonb_path_query('"1.23"', '$.double()'); select jsonb_path_query('"1.23aaa"', '$.double()'); @@ -317,22 +317,22 @@ select jsonb_path_query('"nan"', '$.double()'); select jsonb_path_query('"NaN"', '$.double()'); select jsonb_path_query('"inf"', '$.double()'); select jsonb_path_query('"-inf"', '$.double()'); -select jsonb_path_query('"inf"', '$.double()', silent => true); -select jsonb_path_query('"-inf"', '$.double()', silent => true); +select jsonb_path_query('"inf"', '$.double()', '{}', true); +select jsonb_path_query('"-inf"', '$.double()', '{}', true); select jsonb_path_query('{}', '$.abs()'); select jsonb_path_query('true', '$.floor()'); select jsonb_path_query('"1.2"', '$.ceiling()'); -select jsonb_path_query('{}', '$.abs()', silent => true); -select jsonb_path_query('true', '$.floor()', silent => true); -select jsonb_path_query('"1.2"', '$.ceiling()', silent => true); +select jsonb_path_query('{}', '$.abs()', '{}', true); +select jsonb_path_query('true', '$.floor()', '{}', true); +select jsonb_path_query('"1.2"', '$.ceiling()', '{}', true); select jsonb_path_query('["", "a", "abc", "abcabc"]', '$[*] ? (@ starts with "abc")'); select jsonb_path_query('["", "a", "abc", "abcabc"]', 'strict $ ? (@[*] starts with "abc")'); select jsonb_path_query('["", "a", "abd", "abdabc"]', 'strict $ ? (@[*] starts with "abc")'); select jsonb_path_query('["abc", "abcabc", null, 1]', 'strict $ ? (@[*] starts with "abc")'); select jsonb_path_query('["abc", "abcabc", null, 1]', 'strict $ ? ((@[*] starts with "abc") is unknown)'); -select jsonb_path_query('[[null, 1, "abc", "abcabc"]]', 'lax $ ? (@[*] starts with "abc")'); +--@ select jsonb_path_query('[[null, 1, "abc", "abcabc"]]', 'lax $ ? (@[*] starts with "abc")'); select jsonb_path_query('[[null, 1, "abd", "abdabc"]]', 'lax $ ? ((@[*] starts with "abc") is unknown)'); select jsonb_path_query('[null, 1, "abd", "abdabc"]', 'lax $[*] ? ((@ starts with "abc") is unknown)'); @@ -341,192 +341,192 @@ select jsonb_path_query('[null, 1, "abc", "abd", "aBdC", "abdacb", "babc", "adc\ select jsonb_path_query('[null, 1, "abc", "abd", "aBdC", "abdacb", "babc", "adc\nabc", "ab\nadc"]', 'lax $[*] ? (@ like_regex "^ab.*c" flag "m")'); select jsonb_path_query('[null, 1, "abc", "abd", "aBdC", "abdacb", "babc", "adc\nabc", "ab\nadc"]', 'lax $[*] ? (@ like_regex "^ab.*c" flag "s")'); select jsonb_path_query('[null, 1, "a\b", "a\\b", "^a\\b$"]', 'lax $[*] ? (@ like_regex "a\\b" flag "q")'); -select jsonb_path_query('[null, 1, "a\b", "a\\b", "^a\\b$"]', 'lax $[*] ? (@ like_regex "a\\b" flag "")'); +--@ select jsonb_path_query('[null, 1, "a\b", "a\\b", "^a\\b$"]', 'lax $[*] ? (@ like_regex "a\\b" flag "")'); select jsonb_path_query('[null, 1, "a\b", "a\\b", "^a\\b$"]', 'lax $[*] ? (@ like_regex "^a\\b$" flag "q")'); select jsonb_path_query('[null, 1, "a\b", "a\\b", "^a\\b$"]', 'lax $[*] ? (@ like_regex "^a\\B$" flag "q")'); select jsonb_path_query('[null, 1, "a\b", "a\\b", "^a\\b$"]', 'lax $[*] ? (@ like_regex "^a\\B$" flag "iq")'); -select jsonb_path_query('[null, 1, "a\b", "a\\b", "^a\\b$"]', 'lax $[*] ? (@ like_regex "^a\\b$" flag "")'); - -select jsonb_path_query('null', '$.datetime()'); -select jsonb_path_query('true', '$.datetime()'); -select jsonb_path_query('1', '$.datetime()'); -select jsonb_path_query('[]', '$.datetime()'); -select jsonb_path_query('[]', 'strict $.datetime()'); -select jsonb_path_query('{}', '$.datetime()'); -select jsonb_path_query('"bogus"', '$.datetime()'); -select jsonb_path_query('"12:34"', '$.datetime("aaa")'); -select jsonb_path_query('"aaaa"', '$.datetime("HH24")'); - -select jsonb '"10-03-2017"' @? '$.datetime("dd-mm-yyyy")'; -select jsonb_path_query('"10-03-2017"', '$.datetime("dd-mm-yyyy")'); -select jsonb_path_query('"10-03-2017"', '$.datetime("dd-mm-yyyy").type()'); -select jsonb_path_query('"10-03-2017 12:34"', '$.datetime("dd-mm-yyyy")'); -select jsonb_path_query('"10-03-2017 12:34"', '$.datetime("dd-mm-yyyy").type()'); - -select jsonb_path_query('"10-03-2017 12:34"', ' $.datetime("dd-mm-yyyy HH24:MI").type()'); -select jsonb_path_query('"10-03-2017 12:34 +05:20"', '$.datetime("dd-mm-yyyy HH24:MI TZH:TZM").type()'); -select jsonb_path_query('"12:34:56"', '$.datetime("HH24:MI:SS").type()'); -select jsonb_path_query('"12:34:56 +05:20"', '$.datetime("HH24:MI:SS TZH:TZM").type()'); - -select jsonb_path_query('"10-03-2017T12:34:56"', '$.datetime("dd-mm-yyyy\"T\"HH24:MI:SS")'); -select jsonb_path_query('"10-03-2017t12:34:56"', '$.datetime("dd-mm-yyyy\"T\"HH24:MI:SS")'); -select jsonb_path_query('"10-03-2017 12:34:56"', '$.datetime("dd-mm-yyyy\"T\"HH24:MI:SS")'); - -set time zone '+00'; - -select jsonb_path_query('"10-03-2017 12:34"', '$.datetime("dd-mm-yyyy HH24:MI")'); -select jsonb_path_query('"10-03-2017 12:34"', '$.datetime("dd-mm-yyyy HH24:MI TZH")'); -select jsonb_path_query('"10-03-2017 12:34 +05"', '$.datetime("dd-mm-yyyy HH24:MI TZH")'); -select jsonb_path_query('"10-03-2017 12:34 -05"', '$.datetime("dd-mm-yyyy HH24:MI TZH")'); -select jsonb_path_query('"10-03-2017 12:34 +05:20"', '$.datetime("dd-mm-yyyy HH24:MI TZH:TZM")'); -select jsonb_path_query('"10-03-2017 12:34 -05:20"', '$.datetime("dd-mm-yyyy HH24:MI TZH:TZM")'); -select jsonb_path_query('"12:34"', '$.datetime("HH24:MI")'); -select jsonb_path_query('"12:34"', '$.datetime("HH24:MI TZH")'); -select jsonb_path_query('"12:34 +05"', '$.datetime("HH24:MI TZH")'); -select jsonb_path_query('"12:34 -05"', '$.datetime("HH24:MI TZH")'); -select jsonb_path_query('"12:34 +05:20"', '$.datetime("HH24:MI TZH:TZM")'); -select jsonb_path_query('"12:34 -05:20"', '$.datetime("HH24:MI TZH:TZM")'); - -set time zone '+10'; - -select jsonb_path_query('"10-03-2017 12:34"', '$.datetime("dd-mm-yyyy HH24:MI")'); -select jsonb_path_query('"10-03-2017 12:34"', '$.datetime("dd-mm-yyyy HH24:MI TZH")'); -select jsonb_path_query('"10-03-2017 12:34 +05"', '$.datetime("dd-mm-yyyy HH24:MI TZH")'); -select jsonb_path_query('"10-03-2017 12:34 -05"', '$.datetime("dd-mm-yyyy HH24:MI TZH")'); -select jsonb_path_query('"10-03-2017 12:34 +05:20"', '$.datetime("dd-mm-yyyy HH24:MI TZH:TZM")'); -select jsonb_path_query('"10-03-2017 12:34 -05:20"', '$.datetime("dd-mm-yyyy HH24:MI TZH:TZM")'); -select jsonb_path_query('"12:34"', '$.datetime("HH24:MI")'); -select jsonb_path_query('"12:34"', '$.datetime("HH24:MI TZH")'); -select jsonb_path_query('"12:34 +05"', '$.datetime("HH24:MI TZH")'); -select jsonb_path_query('"12:34 -05"', '$.datetime("HH24:MI TZH")'); -select jsonb_path_query('"12:34 +05:20"', '$.datetime("HH24:MI TZH:TZM")'); -select jsonb_path_query('"12:34 -05:20"', '$.datetime("HH24:MI TZH:TZM")'); - -set time zone default; - -select jsonb_path_query('"2017-03-10"', '$.datetime().type()'); -select jsonb_path_query('"2017-03-10"', '$.datetime()'); -select jsonb_path_query('"2017-03-10 12:34:56"', '$.datetime().type()'); -select jsonb_path_query('"2017-03-10 12:34:56"', '$.datetime()'); -select jsonb_path_query('"2017-03-10 12:34:56+3"', '$.datetime().type()'); -select jsonb_path_query('"2017-03-10 12:34:56+3"', '$.datetime()'); -select jsonb_path_query('"2017-03-10 12:34:56+3:10"', '$.datetime().type()'); -select jsonb_path_query('"2017-03-10 12:34:56+3:10"', '$.datetime()'); -select jsonb_path_query('"2017-03-10T12:34:56+3:10"', '$.datetime()'); -select jsonb_path_query('"2017-03-10t12:34:56+3:10"', '$.datetime()'); -select jsonb_path_query('"12:34:56"', '$.datetime().type()'); -select jsonb_path_query('"12:34:56"', '$.datetime()'); -select jsonb_path_query('"12:34:56+3"', '$.datetime().type()'); -select jsonb_path_query('"12:34:56+3"', '$.datetime()'); -select jsonb_path_query('"12:34:56+3:10"', '$.datetime().type()'); -select jsonb_path_query('"12:34:56+3:10"', '$.datetime()'); - -set time zone '+00'; - --- date comparison -select jsonb_path_query( - '["2017-03-10", "2017-03-11", "2017-03-09", "12:34:56", "01:02:03+04", "2017-03-10 00:00:00", "2017-03-10 12:34:56", "2017-03-10 01:02:03+04", "2017-03-10 03:00:00+03"]', - '$[*].datetime() ? (@ == "10.03.2017".datetime("dd.mm.yyyy"))'); -select jsonb_path_query( - '["2017-03-10", "2017-03-11", "2017-03-09", "12:34:56", "01:02:03+04", "2017-03-10 00:00:00", "2017-03-10 12:34:56", "2017-03-10 01:02:03+04", "2017-03-10 03:00:00+03"]', - '$[*].datetime() ? (@ >= "10.03.2017".datetime("dd.mm.yyyy"))'); -select jsonb_path_query( - '["2017-03-10", "2017-03-11", "2017-03-09", "12:34:56", "01:02:03+04", "2017-03-10 00:00:00", "2017-03-10 12:34:56", "2017-03-10 01:02:03+04", "2017-03-10 03:00:00+03"]', - '$[*].datetime() ? (@ < "10.03.2017".datetime("dd.mm.yyyy"))'); -select jsonb_path_query_tz( - '["2017-03-10", "2017-03-11", "2017-03-09", "12:34:56", "01:02:03+04", "2017-03-10 00:00:00", "2017-03-10 12:34:56", "2017-03-10 01:02:03+04", "2017-03-10 03:00:00+03"]', - '$[*].datetime() ? (@ == "10.03.2017".datetime("dd.mm.yyyy"))'); -select jsonb_path_query_tz( - '["2017-03-10", "2017-03-11", "2017-03-09", "12:34:56", "01:02:03+04", "2017-03-10 00:00:00", "2017-03-10 12:34:56", "2017-03-10 01:02:03+04", "2017-03-10 03:00:00+03"]', - '$[*].datetime() ? (@ >= "10.03.2017".datetime("dd.mm.yyyy"))'); -select jsonb_path_query_tz( - '["2017-03-10", "2017-03-11", "2017-03-09", "12:34:56", "01:02:03+04", "2017-03-10 00:00:00", "2017-03-10 12:34:56", "2017-03-10 01:02:03+04", "2017-03-10 03:00:00+03"]', - '$[*].datetime() ? (@ < "10.03.2017".datetime("dd.mm.yyyy"))'); - --- time comparison -select jsonb_path_query( - '["12:34:00", "12:35:00", "12:36:00", "12:35:00+00", "12:35:00+01", "13:35:00+01", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00+01"]', - '$[*].datetime() ? (@ == "12:35".datetime("HH24:MI"))'); -select jsonb_path_query( - '["12:34:00", "12:35:00", "12:36:00", "12:35:00+00", "12:35:00+01", "13:35:00+01", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00+01"]', - '$[*].datetime() ? (@ >= "12:35".datetime("HH24:MI"))'); -select jsonb_path_query( - '["12:34:00", "12:35:00", "12:36:00", "12:35:00+00", "12:35:00+01", "13:35:00+01", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00+01"]', - '$[*].datetime() ? (@ < "12:35".datetime("HH24:MI"))'); -select jsonb_path_query_tz( - '["12:34:00", "12:35:00", "12:36:00", "12:35:00+00", "12:35:00+01", "13:35:00+01", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00+01"]', - '$[*].datetime() ? (@ == "12:35".datetime("HH24:MI"))'); -select jsonb_path_query_tz( - '["12:34:00", "12:35:00", "12:36:00", "12:35:00+00", "12:35:00+01", "13:35:00+01", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00+01"]', - '$[*].datetime() ? (@ >= "12:35".datetime("HH24:MI"))'); -select jsonb_path_query_tz( - '["12:34:00", "12:35:00", "12:36:00", "12:35:00+00", "12:35:00+01", "13:35:00+01", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00+01"]', - '$[*].datetime() ? (@ < "12:35".datetime("HH24:MI"))'); - --- timetz comparison -select jsonb_path_query( - '["12:34:00+01", "12:35:00+01", "12:36:00+01", "12:35:00+02", "12:35:00-02", "10:35:00", "11:35:00", "12:35:00", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00 +1"]', - '$[*].datetime() ? (@ == "12:35 +1".datetime("HH24:MI TZH"))'); -select jsonb_path_query( - '["12:34:00+01", "12:35:00+01", "12:36:00+01", "12:35:00+02", "12:35:00-02", "10:35:00", "11:35:00", "12:35:00", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00 +1"]', - '$[*].datetime() ? (@ >= "12:35 +1".datetime("HH24:MI TZH"))'); -select jsonb_path_query( - '["12:34:00+01", "12:35:00+01", "12:36:00+01", "12:35:00+02", "12:35:00-02", "10:35:00", "11:35:00", "12:35:00", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00 +1"]', - '$[*].datetime() ? (@ < "12:35 +1".datetime("HH24:MI TZH"))'); -select jsonb_path_query_tz( - '["12:34:00+01", "12:35:00+01", "12:36:00+01", "12:35:00+02", "12:35:00-02", "10:35:00", "11:35:00", "12:35:00", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00 +1"]', - '$[*].datetime() ? (@ == "12:35 +1".datetime("HH24:MI TZH"))'); -select jsonb_path_query_tz( - '["12:34:00+01", "12:35:00+01", "12:36:00+01", "12:35:00+02", "12:35:00-02", "10:35:00", "11:35:00", "12:35:00", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00 +1"]', - '$[*].datetime() ? (@ >= "12:35 +1".datetime("HH24:MI TZH"))'); -select jsonb_path_query_tz( - '["12:34:00+01", "12:35:00+01", "12:36:00+01", "12:35:00+02", "12:35:00-02", "10:35:00", "11:35:00", "12:35:00", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00 +1"]', - '$[*].datetime() ? (@ < "12:35 +1".datetime("HH24:MI TZH"))'); - --- timestamp comparison -select jsonb_path_query( - '["2017-03-10 12:34:00", "2017-03-10 12:35:00", "2017-03-10 12:36:00", "2017-03-10 12:35:00+01", "2017-03-10 13:35:00+01", "2017-03-10 12:35:00-01", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', - '$[*].datetime() ? (@ == "10.03.2017 12:35".datetime("dd.mm.yyyy HH24:MI"))'); -select jsonb_path_query( - '["2017-03-10 12:34:00", "2017-03-10 12:35:00", "2017-03-10 12:36:00", "2017-03-10 12:35:00+01", "2017-03-10 13:35:00+01", "2017-03-10 12:35:00-01", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', - '$[*].datetime() ? (@ >= "10.03.2017 12:35".datetime("dd.mm.yyyy HH24:MI"))'); -select jsonb_path_query( - '["2017-03-10 12:34:00", "2017-03-10 12:35:00", "2017-03-10 12:36:00", "2017-03-10 12:35:00+01", "2017-03-10 13:35:00+01", "2017-03-10 12:35:00-01", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', - '$[*].datetime() ? (@ < "10.03.2017 12:35".datetime("dd.mm.yyyy HH24:MI"))'); -select jsonb_path_query_tz( - '["2017-03-10 12:34:00", "2017-03-10 12:35:00", "2017-03-10 12:36:00", "2017-03-10 12:35:00+01", "2017-03-10 13:35:00+01", "2017-03-10 12:35:00-01", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', - '$[*].datetime() ? (@ == "10.03.2017 12:35".datetime("dd.mm.yyyy HH24:MI"))'); -select jsonb_path_query_tz( - '["2017-03-10 12:34:00", "2017-03-10 12:35:00", "2017-03-10 12:36:00", "2017-03-10 12:35:00+01", "2017-03-10 13:35:00+01", "2017-03-10 12:35:00-01", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', - '$[*].datetime() ? (@ >= "10.03.2017 12:35".datetime("dd.mm.yyyy HH24:MI"))'); -select jsonb_path_query_tz( - '["2017-03-10 12:34:00", "2017-03-10 12:35:00", "2017-03-10 12:36:00", "2017-03-10 12:35:00+01", "2017-03-10 13:35:00+01", "2017-03-10 12:35:00-01", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', - '$[*].datetime() ? (@ < "10.03.2017 12:35".datetime("dd.mm.yyyy HH24:MI"))'); - --- timestamptz comparison -select jsonb_path_query( - '["2017-03-10 12:34:00+01", "2017-03-10 12:35:00+01", "2017-03-10 12:36:00+01", "2017-03-10 12:35:00+02", "2017-03-10 12:35:00-02", "2017-03-10 10:35:00", "2017-03-10 11:35:00", "2017-03-10 12:35:00", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', - '$[*].datetime() ? (@ == "10.03.2017 12:35 +1".datetime("dd.mm.yyyy HH24:MI TZH"))'); -select jsonb_path_query( - '["2017-03-10 12:34:00+01", "2017-03-10 12:35:00+01", "2017-03-10 12:36:00+01", "2017-03-10 12:35:00+02", "2017-03-10 12:35:00-02", "2017-03-10 10:35:00", "2017-03-10 11:35:00", "2017-03-10 12:35:00", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', - '$[*].datetime() ? (@ >= "10.03.2017 12:35 +1".datetime("dd.mm.yyyy HH24:MI TZH"))'); -select jsonb_path_query( - '["2017-03-10 12:34:00+01", "2017-03-10 12:35:00+01", "2017-03-10 12:36:00+01", "2017-03-10 12:35:00+02", "2017-03-10 12:35:00-02", "2017-03-10 10:35:00", "2017-03-10 11:35:00", "2017-03-10 12:35:00", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', - '$[*].datetime() ? (@ < "10.03.2017 12:35 +1".datetime("dd.mm.yyyy HH24:MI TZH"))'); -select jsonb_path_query_tz( - '["2017-03-10 12:34:00+01", "2017-03-10 12:35:00+01", "2017-03-10 12:36:00+01", "2017-03-10 12:35:00+02", "2017-03-10 12:35:00-02", "2017-03-10 10:35:00", "2017-03-10 11:35:00", "2017-03-10 12:35:00", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', - '$[*].datetime() ? (@ == "10.03.2017 12:35 +1".datetime("dd.mm.yyyy HH24:MI TZH"))'); -select jsonb_path_query_tz( - '["2017-03-10 12:34:00+01", "2017-03-10 12:35:00+01", "2017-03-10 12:36:00+01", "2017-03-10 12:35:00+02", "2017-03-10 12:35:00-02", "2017-03-10 10:35:00", "2017-03-10 11:35:00", "2017-03-10 12:35:00", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', - '$[*].datetime() ? (@ >= "10.03.2017 12:35 +1".datetime("dd.mm.yyyy HH24:MI TZH"))'); -select jsonb_path_query_tz( - '["2017-03-10 12:34:00+01", "2017-03-10 12:35:00+01", "2017-03-10 12:36:00+01", "2017-03-10 12:35:00+02", "2017-03-10 12:35:00-02", "2017-03-10 10:35:00", "2017-03-10 11:35:00", "2017-03-10 12:35:00", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', - '$[*].datetime() ? (@ < "10.03.2017 12:35 +1".datetime("dd.mm.yyyy HH24:MI TZH"))'); - --- overflow during comparison -select jsonb_path_query('"1000000-01-01"', '$.datetime() > "2020-01-01 12:00:00".datetime()'::jsonpath); - -set time zone default; +--@ select jsonb_path_query('[null, 1, "a\b", "a\\b", "^a\\b$"]', 'lax $[*] ? (@ like_regex "^a\\b$" flag "")'); + +--@ select jsonb_path_query('null', '$.datetime()'); +--@ select jsonb_path_query('true', '$.datetime()'); +--@ select jsonb_path_query('1', '$.datetime()'); +--@ select jsonb_path_query('[]', '$.datetime()'); +--@ select jsonb_path_query('[]', 'strict $.datetime()'); +--@ select jsonb_path_query('{}', '$.datetime()'); +--@ select jsonb_path_query('"bogus"', '$.datetime()'); +--@ select jsonb_path_query('"12:34"', '$.datetime("aaa")'); +--@ select jsonb_path_query('"aaaa"', '$.datetime("HH24")'); +--@ +--@ select jsonb '"10-03-2017"' @? '$.datetime("dd-mm-yyyy")'; +--@ select jsonb_path_query('"10-03-2017"', '$.datetime("dd-mm-yyyy")'); +--@ select jsonb_path_query('"10-03-2017"', '$.datetime("dd-mm-yyyy").type()'); +--@ select jsonb_path_query('"10-03-2017 12:34"', '$.datetime("dd-mm-yyyy")'); +--@ select jsonb_path_query('"10-03-2017 12:34"', '$.datetime("dd-mm-yyyy").type()'); +--@ +--@ select jsonb_path_query('"10-03-2017 12:34"', ' $.datetime("dd-mm-yyyy HH24:MI").type()'); +--@ select jsonb_path_query('"10-03-2017 12:34 +05:20"', '$.datetime("dd-mm-yyyy HH24:MI TZH:TZM").type()'); +--@ select jsonb_path_query('"12:34:56"', '$.datetime("HH24:MI:SS").type()'); +--@ select jsonb_path_query('"12:34:56 +05:20"', '$.datetime("HH24:MI:SS TZH:TZM").type()'); +--@ +--@ select jsonb_path_query('"10-03-2017T12:34:56"', '$.datetime("dd-mm-yyyy\"T\"HH24:MI:SS")'); +--@ select jsonb_path_query('"10-03-2017t12:34:56"', '$.datetime("dd-mm-yyyy\"T\"HH24:MI:SS")'); +--@ select jsonb_path_query('"10-03-2017 12:34:56"', '$.datetime("dd-mm-yyyy\"T\"HH24:MI:SS")'); +--@ +--@ set time zone '+00'; +--@ +--@ select jsonb_path_query('"10-03-2017 12:34"', '$.datetime("dd-mm-yyyy HH24:MI")'); +--@ select jsonb_path_query('"10-03-2017 12:34"', '$.datetime("dd-mm-yyyy HH24:MI TZH")'); +--@ select jsonb_path_query('"10-03-2017 12:34 +05"', '$.datetime("dd-mm-yyyy HH24:MI TZH")'); +--@ select jsonb_path_query('"10-03-2017 12:34 -05"', '$.datetime("dd-mm-yyyy HH24:MI TZH")'); +--@ select jsonb_path_query('"10-03-2017 12:34 +05:20"', '$.datetime("dd-mm-yyyy HH24:MI TZH:TZM")'); +--@ select jsonb_path_query('"10-03-2017 12:34 -05:20"', '$.datetime("dd-mm-yyyy HH24:MI TZH:TZM")'); +--@ select jsonb_path_query('"12:34"', '$.datetime("HH24:MI")'); +--@ select jsonb_path_query('"12:34"', '$.datetime("HH24:MI TZH")'); +--@ select jsonb_path_query('"12:34 +05"', '$.datetime("HH24:MI TZH")'); +--@ select jsonb_path_query('"12:34 -05"', '$.datetime("HH24:MI TZH")'); +--@ select jsonb_path_query('"12:34 +05:20"', '$.datetime("HH24:MI TZH:TZM")'); +--@ select jsonb_path_query('"12:34 -05:20"', '$.datetime("HH24:MI TZH:TZM")'); +--@ +--@ set time zone '+10'; +--@ +--@ select jsonb_path_query('"10-03-2017 12:34"', '$.datetime("dd-mm-yyyy HH24:MI")'); +--@ select jsonb_path_query('"10-03-2017 12:34"', '$.datetime("dd-mm-yyyy HH24:MI TZH")'); +--@ select jsonb_path_query('"10-03-2017 12:34 +05"', '$.datetime("dd-mm-yyyy HH24:MI TZH")'); +--@ select jsonb_path_query('"10-03-2017 12:34 -05"', '$.datetime("dd-mm-yyyy HH24:MI TZH")'); +--@ select jsonb_path_query('"10-03-2017 12:34 +05:20"', '$.datetime("dd-mm-yyyy HH24:MI TZH:TZM")'); +--@ select jsonb_path_query('"10-03-2017 12:34 -05:20"', '$.datetime("dd-mm-yyyy HH24:MI TZH:TZM")'); +--@ select jsonb_path_query('"12:34"', '$.datetime("HH24:MI")'); +--@ select jsonb_path_query('"12:34"', '$.datetime("HH24:MI TZH")'); +--@ select jsonb_path_query('"12:34 +05"', '$.datetime("HH24:MI TZH")'); +--@ select jsonb_path_query('"12:34 -05"', '$.datetime("HH24:MI TZH")'); +--@ select jsonb_path_query('"12:34 +05:20"', '$.datetime("HH24:MI TZH:TZM")'); +--@ select jsonb_path_query('"12:34 -05:20"', '$.datetime("HH24:MI TZH:TZM")'); +--@ +--@ set time zone default; +--@ +--@ select jsonb_path_query('"2017-03-10"', '$.datetime().type()'); +--@ select jsonb_path_query('"2017-03-10"', '$.datetime()'); +--@ select jsonb_path_query('"2017-03-10 12:34:56"', '$.datetime().type()'); +--@ select jsonb_path_query('"2017-03-10 12:34:56"', '$.datetime()'); +--@ select jsonb_path_query('"2017-03-10 12:34:56+3"', '$.datetime().type()'); +--@ select jsonb_path_query('"2017-03-10 12:34:56+3"', '$.datetime()'); +--@ select jsonb_path_query('"2017-03-10 12:34:56+3:10"', '$.datetime().type()'); +--@ select jsonb_path_query('"2017-03-10 12:34:56+3:10"', '$.datetime()'); +--@ select jsonb_path_query('"2017-03-10T12:34:56+3:10"', '$.datetime()'); +--@ select jsonb_path_query('"2017-03-10t12:34:56+3:10"', '$.datetime()'); +--@ select jsonb_path_query('"12:34:56"', '$.datetime().type()'); +--@ select jsonb_path_query('"12:34:56"', '$.datetime()'); +--@ select jsonb_path_query('"12:34:56+3"', '$.datetime().type()'); +--@ select jsonb_path_query('"12:34:56+3"', '$.datetime()'); +--@ select jsonb_path_query('"12:34:56+3:10"', '$.datetime().type()'); +--@ select jsonb_path_query('"12:34:56+3:10"', '$.datetime()'); +--@ +--@ set time zone '+00'; +--@ +--@ -- date comparison +--@ select jsonb_path_query( +--@ '["2017-03-10", "2017-03-11", "2017-03-09", "12:34:56", "01:02:03+04", "2017-03-10 00:00:00", "2017-03-10 12:34:56", "2017-03-10 01:02:03+04", "2017-03-10 03:00:00+03"]', +--@ '$[*].datetime() ? (@ == "10.03.2017".datetime("dd.mm.yyyy"))'); +--@ select jsonb_path_query( +--@ '["2017-03-10", "2017-03-11", "2017-03-09", "12:34:56", "01:02:03+04", "2017-03-10 00:00:00", "2017-03-10 12:34:56", "2017-03-10 01:02:03+04", "2017-03-10 03:00:00+03"]', +--@ '$[*].datetime() ? (@ >= "10.03.2017".datetime("dd.mm.yyyy"))'); +--@ select jsonb_path_query( +--@ '["2017-03-10", "2017-03-11", "2017-03-09", "12:34:56", "01:02:03+04", "2017-03-10 00:00:00", "2017-03-10 12:34:56", "2017-03-10 01:02:03+04", "2017-03-10 03:00:00+03"]', +--@ '$[*].datetime() ? (@ < "10.03.2017".datetime("dd.mm.yyyy"))'); +--@ select jsonb_path_query_tz( +--@ '["2017-03-10", "2017-03-11", "2017-03-09", "12:34:56", "01:02:03+04", "2017-03-10 00:00:00", "2017-03-10 12:34:56", "2017-03-10 01:02:03+04", "2017-03-10 03:00:00+03"]', +--@ '$[*].datetime() ? (@ == "10.03.2017".datetime("dd.mm.yyyy"))'); +--@ select jsonb_path_query_tz( +--@ '["2017-03-10", "2017-03-11", "2017-03-09", "12:34:56", "01:02:03+04", "2017-03-10 00:00:00", "2017-03-10 12:34:56", "2017-03-10 01:02:03+04", "2017-03-10 03:00:00+03"]', +--@ '$[*].datetime() ? (@ >= "10.03.2017".datetime("dd.mm.yyyy"))'); +--@ select jsonb_path_query_tz( +--@ '["2017-03-10", "2017-03-11", "2017-03-09", "12:34:56", "01:02:03+04", "2017-03-10 00:00:00", "2017-03-10 12:34:56", "2017-03-10 01:02:03+04", "2017-03-10 03:00:00+03"]', +--@ '$[*].datetime() ? (@ < "10.03.2017".datetime("dd.mm.yyyy"))'); +--@ +--@ -- time comparison +--@ select jsonb_path_query( +--@ '["12:34:00", "12:35:00", "12:36:00", "12:35:00+00", "12:35:00+01", "13:35:00+01", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00+01"]', +--@ '$[*].datetime() ? (@ == "12:35".datetime("HH24:MI"))'); +--@ select jsonb_path_query( +--@ '["12:34:00", "12:35:00", "12:36:00", "12:35:00+00", "12:35:00+01", "13:35:00+01", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00+01"]', +--@ '$[*].datetime() ? (@ >= "12:35".datetime("HH24:MI"))'); +--@ select jsonb_path_query( +--@ '["12:34:00", "12:35:00", "12:36:00", "12:35:00+00", "12:35:00+01", "13:35:00+01", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00+01"]', +--@ '$[*].datetime() ? (@ < "12:35".datetime("HH24:MI"))'); +--@ select jsonb_path_query_tz( +--@ '["12:34:00", "12:35:00", "12:36:00", "12:35:00+00", "12:35:00+01", "13:35:00+01", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00+01"]', +--@ '$[*].datetime() ? (@ == "12:35".datetime("HH24:MI"))'); +--@ select jsonb_path_query_tz( +--@ '["12:34:00", "12:35:00", "12:36:00", "12:35:00+00", "12:35:00+01", "13:35:00+01", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00+01"]', +--@ '$[*].datetime() ? (@ >= "12:35".datetime("HH24:MI"))'); +--@ select jsonb_path_query_tz( +--@ '["12:34:00", "12:35:00", "12:36:00", "12:35:00+00", "12:35:00+01", "13:35:00+01", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00+01"]', +--@ '$[*].datetime() ? (@ < "12:35".datetime("HH24:MI"))'); +--@ +--@ -- timetz comparison +--@ select jsonb_path_query( +--@ '["12:34:00+01", "12:35:00+01", "12:36:00+01", "12:35:00+02", "12:35:00-02", "10:35:00", "11:35:00", "12:35:00", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00 +1"]', +--@ '$[*].datetime() ? (@ == "12:35 +1".datetime("HH24:MI TZH"))'); +--@ select jsonb_path_query( +--@ '["12:34:00+01", "12:35:00+01", "12:36:00+01", "12:35:00+02", "12:35:00-02", "10:35:00", "11:35:00", "12:35:00", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00 +1"]', +--@ '$[*].datetime() ? (@ >= "12:35 +1".datetime("HH24:MI TZH"))'); +--@ select jsonb_path_query( +--@ '["12:34:00+01", "12:35:00+01", "12:36:00+01", "12:35:00+02", "12:35:00-02", "10:35:00", "11:35:00", "12:35:00", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00 +1"]', +--@ '$[*].datetime() ? (@ < "12:35 +1".datetime("HH24:MI TZH"))'); +--@ select jsonb_path_query_tz( +--@ '["12:34:00+01", "12:35:00+01", "12:36:00+01", "12:35:00+02", "12:35:00-02", "10:35:00", "11:35:00", "12:35:00", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00 +1"]', +--@ '$[*].datetime() ? (@ == "12:35 +1".datetime("HH24:MI TZH"))'); +--@ select jsonb_path_query_tz( +--@ '["12:34:00+01", "12:35:00+01", "12:36:00+01", "12:35:00+02", "12:35:00-02", "10:35:00", "11:35:00", "12:35:00", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00 +1"]', +--@ '$[*].datetime() ? (@ >= "12:35 +1".datetime("HH24:MI TZH"))'); +--@ select jsonb_path_query_tz( +--@ '["12:34:00+01", "12:35:00+01", "12:36:00+01", "12:35:00+02", "12:35:00-02", "10:35:00", "11:35:00", "12:35:00", "2017-03-10", "2017-03-10 12:35:00", "2017-03-10 12:35:00 +1"]', +--@ '$[*].datetime() ? (@ < "12:35 +1".datetime("HH24:MI TZH"))'); +--@ +--@ -- timestamp comparison +--@ select jsonb_path_query( +--@ '["2017-03-10 12:34:00", "2017-03-10 12:35:00", "2017-03-10 12:36:00", "2017-03-10 12:35:00+01", "2017-03-10 13:35:00+01", "2017-03-10 12:35:00-01", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', +--@ '$[*].datetime() ? (@ == "10.03.2017 12:35".datetime("dd.mm.yyyy HH24:MI"))'); +--@ select jsonb_path_query( +--@ '["2017-03-10 12:34:00", "2017-03-10 12:35:00", "2017-03-10 12:36:00", "2017-03-10 12:35:00+01", "2017-03-10 13:35:00+01", "2017-03-10 12:35:00-01", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', +--@ '$[*].datetime() ? (@ >= "10.03.2017 12:35".datetime("dd.mm.yyyy HH24:MI"))'); +--@ select jsonb_path_query( +--@ '["2017-03-10 12:34:00", "2017-03-10 12:35:00", "2017-03-10 12:36:00", "2017-03-10 12:35:00+01", "2017-03-10 13:35:00+01", "2017-03-10 12:35:00-01", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', +--@ '$[*].datetime() ? (@ < "10.03.2017 12:35".datetime("dd.mm.yyyy HH24:MI"))'); +--@ select jsonb_path_query_tz( +--@ '["2017-03-10 12:34:00", "2017-03-10 12:35:00", "2017-03-10 12:36:00", "2017-03-10 12:35:00+01", "2017-03-10 13:35:00+01", "2017-03-10 12:35:00-01", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', +--@ '$[*].datetime() ? (@ == "10.03.2017 12:35".datetime("dd.mm.yyyy HH24:MI"))'); +--@ select jsonb_path_query_tz( +--@ '["2017-03-10 12:34:00", "2017-03-10 12:35:00", "2017-03-10 12:36:00", "2017-03-10 12:35:00+01", "2017-03-10 13:35:00+01", "2017-03-10 12:35:00-01", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', +--@ '$[*].datetime() ? (@ >= "10.03.2017 12:35".datetime("dd.mm.yyyy HH24:MI"))'); +--@ select jsonb_path_query_tz( +--@ '["2017-03-10 12:34:00", "2017-03-10 12:35:00", "2017-03-10 12:36:00", "2017-03-10 12:35:00+01", "2017-03-10 13:35:00+01", "2017-03-10 12:35:00-01", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', +--@ '$[*].datetime() ? (@ < "10.03.2017 12:35".datetime("dd.mm.yyyy HH24:MI"))'); +--@ +--@ -- timestamptz comparison +--@ select jsonb_path_query( +--@ '["2017-03-10 12:34:00+01", "2017-03-10 12:35:00+01", "2017-03-10 12:36:00+01", "2017-03-10 12:35:00+02", "2017-03-10 12:35:00-02", "2017-03-10 10:35:00", "2017-03-10 11:35:00", "2017-03-10 12:35:00", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', +--@ '$[*].datetime() ? (@ == "10.03.2017 12:35 +1".datetime("dd.mm.yyyy HH24:MI TZH"))'); +--@ select jsonb_path_query( +--@ '["2017-03-10 12:34:00+01", "2017-03-10 12:35:00+01", "2017-03-10 12:36:00+01", "2017-03-10 12:35:00+02", "2017-03-10 12:35:00-02", "2017-03-10 10:35:00", "2017-03-10 11:35:00", "2017-03-10 12:35:00", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', +--@ '$[*].datetime() ? (@ >= "10.03.2017 12:35 +1".datetime("dd.mm.yyyy HH24:MI TZH"))'); +--@ select jsonb_path_query( +--@ '["2017-03-10 12:34:00+01", "2017-03-10 12:35:00+01", "2017-03-10 12:36:00+01", "2017-03-10 12:35:00+02", "2017-03-10 12:35:00-02", "2017-03-10 10:35:00", "2017-03-10 11:35:00", "2017-03-10 12:35:00", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', +--@ '$[*].datetime() ? (@ < "10.03.2017 12:35 +1".datetime("dd.mm.yyyy HH24:MI TZH"))'); +--@ select jsonb_path_query_tz( +--@ '["2017-03-10 12:34:00+01", "2017-03-10 12:35:00+01", "2017-03-10 12:36:00+01", "2017-03-10 12:35:00+02", "2017-03-10 12:35:00-02", "2017-03-10 10:35:00", "2017-03-10 11:35:00", "2017-03-10 12:35:00", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', +--@ '$[*].datetime() ? (@ == "10.03.2017 12:35 +1".datetime("dd.mm.yyyy HH24:MI TZH"))'); +--@ select jsonb_path_query_tz( +--@ '["2017-03-10 12:34:00+01", "2017-03-10 12:35:00+01", "2017-03-10 12:36:00+01", "2017-03-10 12:35:00+02", "2017-03-10 12:35:00-02", "2017-03-10 10:35:00", "2017-03-10 11:35:00", "2017-03-10 12:35:00", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', +--@ '$[*].datetime() ? (@ >= "10.03.2017 12:35 +1".datetime("dd.mm.yyyy HH24:MI TZH"))'); +--@ select jsonb_path_query_tz( +--@ '["2017-03-10 12:34:00+01", "2017-03-10 12:35:00+01", "2017-03-10 12:36:00+01", "2017-03-10 12:35:00+02", "2017-03-10 12:35:00-02", "2017-03-10 10:35:00", "2017-03-10 11:35:00", "2017-03-10 12:35:00", "2017-03-10", "2017-03-11", "12:34:56", "12:34:56+01"]', +--@ '$[*].datetime() ? (@ < "10.03.2017 12:35 +1".datetime("dd.mm.yyyy HH24:MI TZH"))'); +--@ +--@ -- overflow during comparison +--@ select jsonb_path_query('"1000000-01-01"', '$.datetime() > "2020-01-01 12:00:00".datetime()'::jsonpath); +--@ +--@ set time zone default; -- jsonpath operators @@ -537,35 +537,35 @@ SELECT jsonb_path_query_array('[{"a": 1}, {"a": 2}, {}]', 'strict $[*].a'); SELECT jsonb_path_query_array('[{"a": 1}, {"a": 2}]', '$[*].a'); SELECT jsonb_path_query_array('[{"a": 1}, {"a": 2}]', '$[*].a ? (@ == 1)'); SELECT jsonb_path_query_array('[{"a": 1}, {"a": 2}]', '$[*].a ? (@ > 10)'); -SELECT jsonb_path_query_array('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*].a ? (@ > $min && @ < $max)', vars => '{"min": 1, "max": 4}'); -SELECT jsonb_path_query_array('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*].a ? (@ > $min && @ < $max)', vars => '{"min": 3, "max": 4}'); +SELECT jsonb_path_query_array('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*].a ? (@ > $min && @ < $max)', '{"min": 1, "max": 4}'); +SELECT jsonb_path_query_array('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*].a ? (@ > $min && @ < $max)', '{"min": 3, "max": 4}'); SELECT jsonb_path_query_first('[{"a": 1}, {"a": 2}, {}]', 'strict $[*].a'); -SELECT jsonb_path_query_first('[{"a": 1}, {"a": 2}, {}]', 'strict $[*].a', silent => true); +--@ SELECT jsonb_path_query_first('[{"a": 1}, {"a": 2}, {}]', 'strict $[*].a', '{}', true); SELECT jsonb_path_query_first('[{"a": 1}, {"a": 2}]', '$[*].a'); SELECT jsonb_path_query_first('[{"a": 1}, {"a": 2}]', '$[*].a ? (@ == 1)'); SELECT jsonb_path_query_first('[{"a": 1}, {"a": 2}]', '$[*].a ? (@ > 10)'); -SELECT jsonb_path_query_first('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*].a ? (@ > $min && @ < $max)', vars => '{"min": 1, "max": 4}'); -SELECT jsonb_path_query_first('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*].a ? (@ > $min && @ < $max)', vars => '{"min": 3, "max": 4}'); +SELECT jsonb_path_query_first('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*].a ? (@ > $min && @ < $max)', '{"min": 1, "max": 4}'); +SELECT jsonb_path_query_first('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*].a ? (@ > $min && @ < $max)', '{"min": 3, "max": 4}'); SELECT jsonb '[{"a": 1}, {"a": 2}]' @? '$[*].a ? (@ > 1)'; SELECT jsonb '[{"a": 1}, {"a": 2}]' @? '$[*] ? (@.a > 2)'; SELECT jsonb_path_exists('[{"a": 1}, {"a": 2}]', '$[*].a ? (@ > 1)'); -SELECT jsonb_path_exists('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*] ? (@.a > $min && @.a < $max)', vars => '{"min": 1, "max": 4}'); -SELECT jsonb_path_exists('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*] ? (@.a > $min && @.a < $max)', vars => '{"min": 3, "max": 4}'); - -SELECT jsonb_path_match('true', '$', silent => false); -SELECT jsonb_path_match('false', '$', silent => false); -SELECT jsonb_path_match('null', '$', silent => false); -SELECT jsonb_path_match('1', '$', silent => true); -SELECT jsonb_path_match('1', '$', silent => false); -SELECT jsonb_path_match('"a"', '$', silent => false); -SELECT jsonb_path_match('{}', '$', silent => false); -SELECT jsonb_path_match('[true]', '$', silent => false); -SELECT jsonb_path_match('{}', 'lax $.a', silent => false); -SELECT jsonb_path_match('{}', 'strict $.a', silent => false); -SELECT jsonb_path_match('{}', 'strict $.a', silent => true); -SELECT jsonb_path_match('[true, true]', '$[*]', silent => false); +SELECT jsonb_path_exists('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*] ? (@.a > $min && @.a < $max)', '{"min": 1, "max": 4}'); +SELECT jsonb_path_exists('[{"a": 1}, {"a": 2}, {"a": 3}, {"a": 5}]', '$[*] ? (@.a > $min && @.a < $max)', '{"min": 3, "max": 4}'); + +SELECT jsonb_path_match('true', '$', '{}', false); +SELECT jsonb_path_match('false', '$', '{}', false); +SELECT jsonb_path_match('null', '$', '{}', false); +SELECT jsonb_path_match('1', '$', '{}', true); +SELECT jsonb_path_match('1', '$', '{}', false); +SELECT jsonb_path_match('"a"', '$', '{}', false); +SELECT jsonb_path_match('{}', '$', '{}', false); +SELECT jsonb_path_match('[true]', '$', '{}', false); +SELECT jsonb_path_match('{}', 'lax $.a', '{}', false); +SELECT jsonb_path_match('{}', 'strict $.a', '{}', false); +SELECT jsonb_path_match('{}', 'strict $.a', '{}', true); +SELECT jsonb_path_match('[true, true]', '$[*]', '{}', false); SELECT jsonb '[{"a": 1}, {"a": 2}]' @@ '$[*].a > 1'; SELECT jsonb '[{"a": 1}, {"a": 2}]' @@ '$[*].a > 2'; SELECT jsonb_path_match('[{"a": 1}, {"a": 2}]', '$[*].a > 1'); @@ -578,10 +578,10 @@ WITH str(j, num) AS ) SELECT s1.j, s2.j, - jsonb_path_query_first(s1.j, '$.s < $s', vars => s2.j) lt, - jsonb_path_query_first(s1.j, '$.s <= $s', vars => s2.j) le, - jsonb_path_query_first(s1.j, '$.s == $s', vars => s2.j) eq, - jsonb_path_query_first(s1.j, '$.s >= $s', vars => s2.j) ge, - jsonb_path_query_first(s1.j, '$.s > $s', vars => s2.j) gt + jsonb_path_query_first(s1.j, '$.s < $s', s2.j) lt, + jsonb_path_query_first(s1.j, '$.s <= $s', s2.j) le, + jsonb_path_query_first(s1.j, '$.s == $s', s2.j) eq, + jsonb_path_query_first(s1.j, '$.s >= $s', s2.j) ge, + jsonb_path_query_first(s1.j, '$.s > $s', s2.j) gt FROM str s1, str s2 ORDER BY s1.num, s2.num; From 5454f6e2d359037280b795dfe444a0b6cf5303cc Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 5 Dec 2023 13:03:43 +0800 Subject: [PATCH 002/215] refactor(cdc): clean-up cdc backfill code for the create table xx with (..) syntax (#13765) --- e2e_test/source/cdc/cdc.load.slt | 4 - e2e_test/source/cdc/cdc.share_stream.slt | 4 - integration_tests/mysql-cdc/create_source.sql | 22 +- src/common/src/session_config/mod.rs | 4 - src/compute/tests/cdc_tests.rs | 132 +++++++---- .../src/source/mock_external_table.rs | 6 - .../tests/testdata/input/create_source.yaml | 4 - .../tests/testdata/output/create_source.yaml | 4 - src/frontend/src/handler/create_source.rs | 11 +- src/frontend/src/handler/create_table.rs | 65 +----- .../src/executor/backfill/cdc/cdc_backfill.rs | 101 ++------- src/stream/src/executor/backfill/cdc/mod.rs | 2 - src/stream/src/executor/backfill/cdc/state.rs | 207 ++---------------- .../executor/source/state_table_handler.rs | 33 +-- .../src/from_proto/source/trad_source.rs | 69 +----- src/stream/src/from_proto/stream_cdc_scan.rs | 4 +- 16 files changed, 157 insertions(+), 515 deletions(-) diff --git a/e2e_test/source/cdc/cdc.load.slt b/e2e_test/source/cdc/cdc.load.slt index 4cfacf120648e..ce0926339c963 100644 --- a/e2e_test/source/cdc/cdc.load.slt +++ b/e2e_test/source/cdc/cdc.load.slt @@ -1,9 +1,5 @@ # CDC source basic test -# enable cdc backfill in ci -statement ok -set cdc_backfill='true'; - statement ok create table products ( id INT, name STRING, diff --git a/e2e_test/source/cdc/cdc.share_stream.slt b/e2e_test/source/cdc/cdc.share_stream.slt index 1e0e4637d5017..5a6bfc9a9f90d 100644 --- a/e2e_test/source/cdc/cdc.share_stream.slt +++ b/e2e_test/source/cdc/cdc.share_stream.slt @@ -11,10 +11,6 @@ mysql --protocol=tcp -u root mytest < e2e_test/source/cdc/mysql_create.sql system ok mysql --protocol=tcp -u root mytest < e2e_test/source/cdc/mysql_init_data.sql -# enable cdc backfill in ci -statement ok -set cdc_backfill='true'; - # create a cdc source job, which format fixed to `FORMAT PLAIN ENCODE JSON` statement ok create source mysql_mytest with ( diff --git a/integration_tests/mysql-cdc/create_source.sql b/integration_tests/mysql-cdc/create_source.sql index 62a0d57c699a2..3f5480ddaa219 100644 --- a/integration_tests/mysql-cdc/create_source.sql +++ b/integration_tests/mysql-cdc/create_source.sql @@ -1,4 +1,13 @@ -set cdc_backfill='true'; + +create source mysql_mydb with ( + connector = 'mysql-cdc', + hostname = 'mysql', + port = '3306', + username = 'root', + password = '123456', + database.name = 'mydb', + server.id = '2' +); CREATE TABLE lineitem_rw ( L_ORDERKEY BIGINT, @@ -18,13 +27,4 @@ CREATE TABLE lineitem_rw ( L_SHIPMODE VARCHAR, L_COMMENT VARCHAR, PRIMARY KEY(L_ORDERKEY, L_LINENUMBER) -) WITH ( - connector = 'mysql-cdc', - hostname = 'mysql', - port = '3306', - username = 'root', - password = '123456', - database.name = 'mydb', - table.name = 'lineitem', - server.id = '2' -); \ No newline at end of file +) FROM mysql_mydb TABLE 'mydb.lineitem'; \ No newline at end of file diff --git a/src/common/src/session_config/mod.rs b/src/common/src/session_config/mod.rs index 6e45acd5c378c..82c5bd0a6f120 100644 --- a/src/common/src/session_config/mod.rs +++ b/src/common/src/session_config/mod.rs @@ -220,10 +220,6 @@ pub struct ConfigMap { #[parameter(default = ConfigNonZeroU64::default())] streaming_rate_limit: ConfigNonZeroU64, - /// Enable backfill for CDC table to allow lock-free and incremental snapshot - #[parameter(default = false)] - cdc_backfill: bool, - /// Cache policy for partition cache in streaming over window. /// Can be "full", "recent", "recent_first_n" or "recent_last_n". #[parameter(default = OverWindowCachePolicy::default(), rename = "rw_streaming_over_window_cache_policy")] diff --git a/src/compute/tests/cdc_tests.rs b/src/compute/tests/cdc_tests.rs index b4baaee5465c4..973ccb1390704 100644 --- a/src/compute/tests/cdc_tests.rs +++ b/src/compute/tests/cdc_tests.rs @@ -17,6 +17,7 @@ use std::collections::{HashMap, HashSet}; use std::marker::PhantomData; +use std::str::FromStr; use std::sync::atomic::AtomicU64; use std::sync::Arc; use std::time::Duration; @@ -25,10 +26,9 @@ use futures::stream::StreamExt; use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_batch::executor::{Executor as BatchExecutor, RowSeqScanExecutor, ScanRange}; -use risingwave_common::array::{ - Array, ArrayBuilder, StreamChunk, StreamChunkTestExt, Utf8ArrayBuilder, -}; +use risingwave_common::array::{Array, ArrayBuilder, DataChunk, Op, StreamChunk, Utf8ArrayBuilder}; use risingwave_common::catalog::{ColumnDesc, ColumnId, ConflictBehavior, Field, Schema, TableId}; +use risingwave_common::types::{Datum, JsonbVal}; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_connector::source::cdc::{CdcSplitBase, DebeziumCdcSplit, MySqlCdcSplit}; use risingwave_connector::source::external::{ @@ -38,15 +38,15 @@ use risingwave_connector::source::{MockExternalTableReader, SplitImpl}; use risingwave_hummock_sdk::to_committed_batch_query_epoch; use risingwave_storage::memory::MemoryStateStore; use risingwave_storage::table::batch_table::storage_table::StorageTable; +use risingwave_stream::common::table::state_table::StateTable; use risingwave_stream::error::StreamResult; use risingwave_stream::executor::external::ExternalStorageTable; use risingwave_stream::executor::monitor::StreamingMetrics; use risingwave_stream::executor::test_utils::MockSource; use risingwave_stream::executor::{ - default_source_internal_table, expect_first_barrier, ActorContext, Barrier, - BoxedExecutor as StreamBoxedExecutor, BoxedMessageStream, CdcBackfillExecutor, Executor, - ExecutorInfo, MaterializeExecutor, Message, Mutation, PkIndices, PkIndicesRef, - SourceStateTableHandler, StreamExecutorError, + expect_first_barrier, ActorContext, Barrier, BoxedExecutor as StreamBoxedExecutor, + BoxedMessageStream, CdcBackfillExecutor, Executor, ExecutorInfo, MaterializeExecutor, Message, + Mutation, PkIndices, PkIndicesRef, StreamExecutorError, }; // mock upstream binlog offset starting from "1.binlog, pos=0" @@ -156,11 +156,10 @@ async fn test_cdc_backfill() -> StreamResult<()> { let table_id = TableId::new(1002); let schema = Schema::new(vec![ - Field::unnamed(DataType::Int64), // primary key - Field::unnamed(DataType::Float64), - Field::unnamed(DataType::Varchar), + Field::unnamed(DataType::Jsonb), // payload + Field::unnamed(DataType::Varchar), // _rw_offset ]); - let column_ids = vec![0.into(), 1.into(), 2.into()]; + let column_ids = vec![0.into(), 1.into()]; let pk_indices = vec![0]; @@ -172,7 +171,6 @@ async fn test_cdc_backfill() -> StreamResult<()> { MockOffsetGenExecutor::new(Box::new(source), schema.clone(), pk_indices.clone()); let binlog_file = String::from("1.binlog"); - // mock binlog watermarks for backfill // initial low watermark: 1.binlog, pos=2 and expected behaviors: // - ignore events before (1.binlog, pos=2); @@ -186,25 +184,48 @@ async fn test_cdc_backfill() -> StreamResult<()> { ]; let table_name = SchemaTableName::new("mock_table".to_string(), "public".to_string()); + let table_schema = Schema::new(vec![ + Field::with_name(DataType::Int64, "id"), // primary key + Field::with_name(DataType::Float64, "price"), + ]); let external_table = ExternalStorageTable::new( table_id, table_name, ExternalTableReaderImpl::Mock(MockExternalTableReader::new(binlog_watermarks)), - schema.clone(), + table_schema.clone(), vec![OrderType::ascending()], pk_indices, vec![0, 1], ); - let source_state_handler = SourceStateTableHandler::from_table_catalog( - &default_source_internal_table(0x2333), - MemoryStateStore::new(), + let actor_id = 0x1a; + + let state_schema = Schema::new(vec![ + Field::with_name(DataType::Varchar, "split_id"), + Field::with_name(DataType::Int64, "id"), // pk + Field::with_name(DataType::Boolean, "backfill_finished"), + Field::with_name(DataType::Int64, "row_count"), + Field::with_name(DataType::Jsonb, "cdc_offset"), + ]); + + let column_descs = vec![ + ColumnDesc::unnamed(ColumnId::from(0), state_schema[0].data_type.clone()), + ColumnDesc::unnamed(ColumnId::from(1), state_schema[1].data_type.clone()), + ColumnDesc::unnamed(ColumnId::from(2), state_schema[2].data_type.clone()), + ColumnDesc::unnamed(ColumnId::from(3), state_schema[3].data_type.clone()), + ColumnDesc::unnamed(ColumnId::from(4), state_schema[4].data_type.clone()), + ]; + + let state_table = StateTable::new_without_distribution( + memory_state_store.clone(), + TableId::from(0x42), + column_descs.clone(), + vec![OrderType::ascending()], + vec![0_usize], ) .await; - - let actor_id = 0x1a; let info = ExecutorInfo { - schema: schema.clone(), + schema: table_schema.clone(), pk_indices: vec![0], identity: "CdcBackfillExecutor".to_string(), }; @@ -213,12 +234,10 @@ async fn test_cdc_backfill() -> StreamResult<()> { info, external_table, Box::new(mock_offset_executor), - vec![0, 1, 2], + vec![0, 1], None, Arc::new(StreamingMetrics::unused()), - None, - Some(source_state_handler), - false, + state_table, 4, // 4 rows in a snapshot chunk ); @@ -237,23 +256,38 @@ async fn test_cdc_backfill() -> StreamResult<()> { .boxed() .execute(); - // push upstream chunks - let stream_chunk1 = StreamChunk::from_pretty( - " I F - + 1 10.01 - + 2 2.02 - + 3 3.03 // binlog pos=2 - + 4 4.04 - + 5 5.05 // binlog pos=4 - + 6 6.06", - ); - let stream_chunk2 = StreamChunk::from_pretty( - " I F - + 6 10.08 - + 199 40.5 - + 978 72.6 - + 134 41.7", - ); + let chunk1_payload = vec![ + r#"{ "payload": { "before": null, "after": { "id": 1, "price": 10.01}, "source": { "version": "1.9.7.Final", "connector": "mysql", "name": "RW_CDC_1002"}, "op": "r", "ts_ms": 1695277757017, "transaction": null } }"#, + r#"{ "payload": { "before": null, "after": { "id": 2, "price": 2.02}, "source": { "version": "1.9.7.Final", "connector": "mysql", "name": "RW_CDC_1002"}, "op": "r", "ts_ms": 1695277757017, "transaction": null } }"#, + r#"{ "payload": { "before": null, "after": { "id": 3, "price": 3.03}, "source": { "version": "1.9.7.Final", "connector": "mysql", "name": "RW_CDC_1002"}, "op": "r", "ts_ms": 1695277757017, "transaction": null } }"#, + r#"{ "payload": { "before": null, "after": { "id": 4, "price": 4.04}, "source": { "version": "1.9.7.Final", "connector": "mysql", "name": "RW_CDC_1002"}, "op": "r", "ts_ms": 1695277757017, "transaction": null } }"#, + r#"{ "payload": { "before": null, "after": { "id": 5, "price": 5.05}, "source": { "version": "1.9.7.Final", "connector": "mysql", "name": "RW_CDC_1002"}, "op": "r", "ts_ms": 1695277757017, "transaction": null } }"#, + r#"{ "payload": { "before": null, "after": { "id": 6, "price": 6.06}, "source": { "version": "1.9.7.Final", "connector": "mysql", "name": "RW_CDC_1002"}, "op": "r", "ts_ms": 1695277757017, "transaction": null } }"#, + ]; + + let chunk2_payload = vec![ + r#"{ "payload": { "before": null, "after": { "id": 6, "price": 10.08}, "source": { "version": "1.9.7.Final", "connector": "mysql", "name": "RW_CDC_1002"}, "op": "r", "ts_ms": 1695277757017, "transaction": null } }"#, + r#"{ "payload": { "before": null, "after": { "id": 199, "price": 40.5}, "source": { "version": "1.9.7.Final", "connector": "mysql", "name": "RW_CDC_1002"}, "op": "r", "ts_ms": 1695277757017, "transaction": null } }"#, + r#"{ "payload": { "before": null, "after": { "id": 978, "price": 72.6}, "source": { "version": "1.9.7.Final", "connector": "mysql", "name": "RW_CDC_1002"}, "op": "r", "ts_ms": 1695277757017, "transaction": null } }"#, + r#"{ "payload": { "before": null, "after": { "id": 134, "price": 41.7}, "source": { "version": "1.9.7.Final", "connector": "mysql", "name": "RW_CDC_1002"}, "op": "r", "ts_ms": 1695277757017, "transaction": null } }"#, + ]; + + let chunk1_datums: Vec = chunk1_payload + .into_iter() + .map(|s| Some(JsonbVal::from_str(s).unwrap().into())) + .collect_vec(); + + let chunk2_datums: Vec = chunk2_payload + .into_iter() + .map(|s| Some(JsonbVal::from_str(s).unwrap().into())) + .collect_vec(); + + let chunk_schema = Schema::new(vec![ + Field::unnamed(DataType::Jsonb), // payload + ]); + + let stream_chunk1 = create_stream_chunk(chunk1_datums, &chunk_schema); + let stream_chunk2 = create_stream_chunk(chunk2_datums, &chunk_schema); // The first barrier let curr_epoch = 11; @@ -311,8 +345,8 @@ async fn test_cdc_backfill() -> StreamResult<()> { // scan the final result of the mv table let column_descs = vec![ - ColumnDesc::unnamed(ColumnId::from(0), schema[0].data_type.clone()), - ColumnDesc::unnamed(ColumnId::from(1), schema[1].data_type.clone()), + ColumnDesc::unnamed(ColumnId::from(0), table_schema[0].data_type.clone()), + ColumnDesc::unnamed(ColumnId::from(1), table_schema[1].data_type.clone()), ]; let value_indices = (0..column_descs.len()).collect_vec(); // Since we have not polled `Materialize`, we cannot scan anything from this table @@ -345,6 +379,20 @@ async fn test_cdc_backfill() -> StreamResult<()> { Ok(()) } +fn create_stream_chunk(datums: Vec, schema: &Schema) -> StreamChunk { + let mut builders = schema.create_array_builders(8); + for datum in &datums { + builders[0].append(datum); + } + let columns = builders + .into_iter() + .map(|builder| builder.finish().into()) + .collect(); + + let ops = vec![Op::Insert; datums.len()]; + StreamChunk::from_parts(ops, DataChunk::new(columns, datums.len())) +} + async fn consume_message_stream(mut stream: BoxedMessageStream) -> StreamResult<()> { while let Some(message) = stream.next().await { let message = message?; diff --git a/src/connector/src/source/mock_external_table.rs b/src/connector/src/source/mock_external_table.rs index 7224a32b9e571..24c43acfff769 100644 --- a/src/connector/src/source/mock_external_table.rs +++ b/src/connector/src/source/mock_external_table.rs @@ -48,33 +48,27 @@ impl MockExternalTableReader { let snap0 = vec![OwnedRow::new(vec![ Some(ScalarImpl::Int64(1)), Some(ScalarImpl::Float64(1.0001.into())), - None, ])]; let snap1 = vec![ OwnedRow::new(vec![ Some(ScalarImpl::Int64(1)), Some(ScalarImpl::Float64(10.01.into())), - None, ]), OwnedRow::new(vec![ Some(ScalarImpl::Int64(2)), Some(ScalarImpl::Float64(2.02.into())), - None, ]), OwnedRow::new(vec![ Some(ScalarImpl::Int64(5)), Some(ScalarImpl::Float64(1.0005.into())), - None, ]), OwnedRow::new(vec![ Some(ScalarImpl::Int64(6)), Some(ScalarImpl::Float64(1.0006.into())), - None, ]), OwnedRow::new(vec![ Some(ScalarImpl::Int64(8)), Some(ScalarImpl::Float64(1.0008.into())), - None, ]), ]; diff --git a/src/frontend/planner_test/tests/testdata/input/create_source.yaml b/src/frontend/planner_test/tests/testdata/input/create_source.yaml index 32fd9c806ae30..be313f83b14da 100644 --- a/src/frontend/planner_test/tests/testdata/input/create_source.yaml +++ b/src/frontend/planner_test/tests/testdata/input/create_source.yaml @@ -51,8 +51,6 @@ ) from mysql_mydb table 'mydb.t1'; expected_outputs: - explain_output - with_config_map: - CDC_BACKFILL: 'true' - id: create_source_with_cdc_backfill sql: | create source mysql_mydb with ( @@ -71,5 +69,3 @@ ) from mysql_mydb table 'mydb.t1'; expected_outputs: - explain_output - with_config_map: - CDC_BACKFILL: 'true' diff --git a/src/frontend/planner_test/tests/testdata/output/create_source.yaml b/src/frontend/planner_test/tests/testdata/output/create_source.yaml index 2aeea6f914d2b..9106ce7af8a72 100644 --- a/src/frontend/planner_test/tests/testdata/output/create_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/create_source.yaml @@ -63,8 +63,6 @@ ) from mysql_mydb table 'mydb.t1'; explain_output: | LogicalCdcScan { table: mydb.t1, columns: [v1, v2] } - with_config_map: - CDC_BACKFILL: 'true' - id: create_source_with_cdc_backfill sql: | create source mysql_mydb with ( @@ -89,5 +87,3 @@ └─StreamExchange { dist: HashShard(v1) } └─StreamDml { columns: [v1, v2] } └─StreamSource - with_config_map: - CDC_BACKFILL: 'true' diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 7050d0b10ccfb..83facac112d05 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -39,6 +39,7 @@ use risingwave_connector::source::cdc::{ MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR, }; use risingwave_connector::source::datagen::DATAGEN_CONNECTOR; +use risingwave_connector::source::external::CdcTableType; use risingwave_connector::source::nexmark::source::{get_event_data_types_with_names, EventType}; use risingwave_connector::source::test_source::TEST_CONNECTOR; use risingwave_connector::source::{ @@ -1101,8 +1102,11 @@ pub async fn handle_create_source( let sql_pk_names = bind_sql_pk_names(&stmt.columns, &stmt.constraints)?; // gated the feature with a session variable - let create_cdc_source_job = - is_cdc_connector(&with_properties) && session.config().cdc_backfill(); + let create_cdc_source_job = if is_cdc_connector(&with_properties) { + CdcTableType::from_properties(&with_properties).can_backfill() + } else { + false + }; let (columns_from_resolve_source, source_info) = bind_columns_from_source( &session, @@ -1301,9 +1305,6 @@ pub mod tests { "CREATE SOURCE t2 WITH (connector = 'mysql-cdc') FORMAT PLAIN ENCODE JSON".to_string(); let frontend = LocalFrontend::new(Default::default()).await; let session = frontend.session_ref(); - session - .set_config("cdc_backfill", "true".to_string()) - .unwrap(); frontend .run_sql_with_session(session.clone(), sql) diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 5bfa3240e7f6b..1c1e5573f6010 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -29,10 +29,7 @@ use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_common::util::value_encoding::DatumToProtoExt; use risingwave_connector::source; -use risingwave_connector::source::cdc::{CDC_SNAPSHOT_BACKFILL, CDC_SNAPSHOT_MODE_KEY}; -use risingwave_connector::source::external::{ - CdcTableType, DATABASE_NAME_KEY, SCHEMA_NAME_KEY, TABLE_NAME_KEY, -}; +use risingwave_connector::source::external::{DATABASE_NAME_KEY, SCHEMA_NAME_KEY, TABLE_NAME_KEY}; use risingwave_pb::catalog::source::OptionalAssociatedTableId; use risingwave_pb::catalog::{PbSource, PbTable, StreamSourceInfo, WatermarkDesc}; use risingwave_pb::ddl_service::TableJobType; @@ -472,7 +469,7 @@ pub(crate) async fn gen_create_table_plan_with_source( let sql_pk_names = bind_sql_pk_names(&column_defs, &constraints)?; - let (columns_from_resolve_source, mut source_info) = + let (columns_from_resolve_source, source_info) = bind_columns_from_source(context.session_ctx(), &source_schema, &properties, false).await?; let columns_from_sql = bind_sql_columns(&column_defs)?; @@ -526,43 +523,6 @@ pub(crate) async fn gen_create_table_plan_with_source( .into()); } - let cdc_table_type = CdcTableType::from_properties(&properties); - if cdc_table_type.can_backfill() && context.session_ctx().config().cdc_backfill() { - // debezium connector will only consume changelogs from latest offset on this mode - properties.insert(CDC_SNAPSHOT_MODE_KEY.into(), CDC_SNAPSHOT_BACKFILL.into()); - - let pk_column_indices = { - let mut id_to_idx = HashMap::new(); - columns.iter().enumerate().for_each(|(idx, c)| { - id_to_idx.insert(c.column_id(), idx); - }); - // pk column id must exist in table columns. - pk_column_ids - .iter() - .map(|c| id_to_idx.get(c).copied().unwrap()) - .collect_vec() - }; - let table_pk = pk_column_indices - .iter() - .map(|idx| ColumnOrder::new(*idx, OrderType::ascending())) - .collect(); - - let cdc_table_desc = CdcTableDesc { - table_id: TableId::placeholder(), - source_id: TableId::placeholder(), - external_table_name: "".to_string(), - pk: table_pk, - columns: columns.iter().map(|c| c.column_desc.clone()).collect(), - stream_key: pk_column_indices, - value_indices: (0..columns.len()).collect_vec(), - connect_properties: Default::default(), - }; - - tracing::debug!(?cdc_table_desc, "create table with source w/ backfill"); - // save external table info to `source_info` - source_info.external_table = Some(cdc_table_desc.to_protobuf()); - } - gen_table_plan_inner( context.into(), table_name, @@ -573,7 +533,6 @@ pub(crate) async fn gen_create_table_plan_with_source( Some(source_info), definition, watermark_descs, - Some(cdc_table_type), append_only, Some(col_id_gen.into_version()), ) @@ -652,7 +611,6 @@ pub(crate) fn gen_create_table_plan_without_bind( None, definition, watermark_descs, - None, append_only, version, ) @@ -669,7 +627,6 @@ fn gen_table_plan_inner( source_info: Option, definition: String, watermark_descs: Vec, - cdc_table_type: Option, append_only: bool, version: Option, /* TODO: this should always be `Some` if we support `ALTER * TABLE` for `CREATE TABLE AS`. */ @@ -693,20 +650,10 @@ fn gen_table_plan_inner( database_id, name: name.clone(), row_id_index: row_id_index.map(|i| i as _), - columns: { - let mut source_columns = columns.clone(); - if let Some(t) = cdc_table_type - && t.can_backfill() - { - // Append the offset column to be used in the cdc backfill - let offset_column = ColumnCatalog::offset_column(); - source_columns.push(offset_column); - } - source_columns - .iter() - .map(|column| column.to_protobuf()) - .collect_vec() - }, + columns: columns + .iter() + .map(|column| column.to_protobuf()) + .collect_vec(), pk_column_ids: pk_column_ids.iter().map(Into::into).collect_vec(), properties: with_options.into_inner().into_iter().collect(), info: Some(source_info), diff --git a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs index f82c6afe41077..347cfb7f0efbe 100644 --- a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs +++ b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs @@ -15,7 +15,6 @@ use std::pin::{pin, Pin}; use std::sync::Arc; -use anyhow::anyhow; use either::Either; use futures::stream::select_with_strategy; use futures::{pin_mut, stream, StreamExt}; @@ -31,13 +30,11 @@ use risingwave_connector::parser::{ SourceStreamChunkBuilder, SpecificParserConfig, }; use risingwave_connector::source::external::CdcOffset; -use risingwave_connector::source::{SourceColumnDesc, SourceContext, SplitMetaData}; +use risingwave_connector::source::{SourceColumnDesc, SourceContext}; use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; -use crate::executor::backfill::cdc::state::{ - CdcBackfillStateImpl, MultiBackfillState, SingleBackfillState, -}; +use crate::executor::backfill::cdc::state::CdcBackfillState; use crate::executor::backfill::upstream_table::external::ExternalStorageTable; use crate::executor::backfill::upstream_table::snapshot::{ SnapshotReadArgs, UpstreamTableRead, UpstreamTableReader, @@ -48,8 +45,7 @@ use crate::executor::backfill::utils::{ use crate::executor::monitor::StreamingMetrics; use crate::executor::{ expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, - ExecutorInfo, Message, Mutation, PkIndicesRef, SourceStateTableHandler, StreamExecutorError, - StreamExecutorResult, + ExecutorInfo, Message, PkIndicesRef, StreamExecutorError, StreamExecutorResult, }; use crate::task::CreateMviewProgress; @@ -70,13 +66,8 @@ pub struct CdcBackfillExecutor { /// User may select a subset of columns from the upstream table. output_indices: Vec, - /// State table of the Source executor - source_state_handler: Option>, - - shared_cdc_source: bool, - /// State table of the CdcBackfill executor - state_table: Option>, + state_table: StateTable, progress: Option, @@ -95,9 +86,7 @@ impl CdcBackfillExecutor { output_indices: Vec, progress: Option, metrics: Arc, - state_table: Option>, - source_state_handler: Option>, - shared_cdc_source: bool, + state_table: StateTable, chunk_size: usize, ) -> Self { Self { @@ -106,8 +95,6 @@ impl CdcBackfillExecutor { external_table, upstream, output_indices, - source_state_handler, - shared_cdc_source, state_table, progress, metrics, @@ -121,7 +108,6 @@ impl CdcBackfillExecutor { let pk_in_output_indices = self.external_table.pk_in_output_indices().unwrap(); let pk_order = self.external_table.pk_order_types().to_vec(); - let shared_cdc_source = self.shared_cdc_source; let upstream_table_id = self.external_table.table_id().table_id; let upstream_table_schema = self.external_table.schema().clone(); let upstream_table_reader = UpstreamTableReader::new(self.external_table); @@ -139,71 +125,15 @@ impl CdcBackfillExecutor { // Check whether this parallelism has been assigned splits, // if not, we should bypass the backfill directly. - let mut state_impl = if shared_cdc_source { - assert!( - self.state_table.is_some(), - "expect state table for shared cdc source" - ); - CdcBackfillStateImpl::MultiTable(MultiBackfillState::new( - upstream_table_id, - self.state_table.unwrap(), - pk_in_output_indices.len() + METADATA_STATE_LEN, - )) - } else if let Some(mutation) = first_barrier.mutation.as_ref() - && let Mutation::Add { splits, .. } = mutation.as_ref() - { - tracing::info!(?mutation, ?shared_cdc_source, "got first barrier"); - - assert!( - self.source_state_handler.is_some(), - "expect source state handler" - ); - - // We can assume for cdc table, the parallism of the fragment must be 1 - match splits.get(&self.actor_ctx.id) { - None => { - unreachable!("expect to receive the cdc split, please check the parallelism of the fragment") - } - Some(splits) => { - if splits.is_empty() { - tracing::info!(?splits, "got empty cdc split, bypass the backfill"); - // The first barrier message should be propagated. - yield Message::Barrier(first_barrier); - #[for_await] - for msg in upstream { - if let Some(msg) = mapping_message(msg?, &self.output_indices) { - yield msg; - } - } - // exit the executor - return Ok(()); - } - - let split = splits.iter().exactly_one().map_err(|_err| { - StreamExecutorError::from(anyhow!( - "expect only one cdc split for table {}", - upstream_table_id - )) - })?; - CdcBackfillStateImpl::SingleTable(SingleBackfillState::new( - self.source_state_handler.unwrap(), - upstream_table_id, - split.id(), - split.clone(), - )) - } - } - } else { - unreachable!("backfilled cdc source init fail") - }; + let mut state_impl = CdcBackfillState::new( + upstream_table_id, + self.state_table, + pk_in_output_indices.len() + METADATA_STATE_LEN, + ); - let mut upstream = if shared_cdc_source { - transform_upstream(upstream, &upstream_table_schema) - .boxed() - .peekable() - } else { - upstream.peekable() - }; + let mut upstream = transform_upstream(upstream, &upstream_table_schema) + .boxed() + .peekable(); state_impl.init_epoch(first_barrier.epoch); @@ -228,10 +158,10 @@ impl CdcBackfillExecutor { tracing::info!( upstream_table_id, - shared_cdc_source, ?current_pk_pos, is_finished = state.is_finished, snapshot_row_count = total_snapshot_row_count, + chunk_size = self.chunk_size, "start cdc backfill" ); @@ -463,7 +393,8 @@ impl CdcBackfillExecutor { Some(get_new_pos(&chunk, &pk_in_output_indices)); tracing::trace!( - "current backfill progress: {:?}", + "got a snapshot chunk: len {}, current_pk_pos {:?}", + chunk.cardinality(), current_pk_pos ); let chunk_cardinality = chunk.cardinality() as u64; diff --git a/src/stream/src/executor/backfill/cdc/mod.rs b/src/stream/src/executor/backfill/cdc/mod.rs index 5061211f28c6a..3f0c48879e544 100644 --- a/src/stream/src/executor/backfill/cdc/mod.rs +++ b/src/stream/src/executor/backfill/cdc/mod.rs @@ -14,5 +14,3 @@ pub mod cdc_backfill; mod state; - -pub use state::BACKFILL_STATE_KEY_SUFFIX; diff --git a/src/stream/src/executor/backfill/cdc/state.rs b/src/stream/src/executor/backfill/cdc/state.rs index f62504415c9d1..ef46b02b222e2 100644 --- a/src/stream/src/executor/backfill/cdc/state.rs +++ b/src/stream/src/executor/backfill/cdc/state.rs @@ -13,33 +13,15 @@ // limitations under the License. use anyhow::anyhow; -use maplit::hashmap; use risingwave_common::row; use risingwave_common::row::{OwnedRow, Row}; -use risingwave_common::types::{Datum, JsonbVal, ScalarImpl, ScalarRefImpl}; +use risingwave_common::types::{Datum, JsonbVal, ScalarImpl}; use risingwave_common::util::epoch::EpochPair; -use risingwave_connector::source::external::{CdcOffset, DebeziumOffset, DebeziumSourceOffset}; -use risingwave_connector::source::{SplitId, SplitImpl, SplitMetaData}; +use risingwave_connector::source::external::CdcOffset; use risingwave_storage::StateStore; -use serde_json::Value; use crate::common::table::state_table::StateTable; -use crate::executor::{SourceStateTableHandler, StreamExecutorResult}; - -/// Depending on how the table is created, we have two scenarios for CDC Backfill: -/// 1. `CREATE TABLE xx WITH ("connector"= 'mysql-cdc', "database.name"='mydb', "table.name"='t1')` -/// In this case, the cdc backfill executor will wraps the source executor, and maintain its state -/// (a finish flag) in the source state table. -/// -/// -/// 2. `CREATE TABLE xx FROM source TABLE 'mydb.t1'` -/// In this case, we can have multiple Table jobs sharing a single cdc Source job. -/// The cdc backfill executor will be an instance of the `StreamScan` operator and has its own state table -/// schema: `table_id | backfill_finished | row_count | cdc_offset` -pub enum CdcBackfillStateImpl { - SingleTable(SingleBackfillState), - MultiTable(MultiBackfillState), -} +use crate::executor::StreamExecutorResult; #[derive(Debug, Default)] pub struct CdcStateRecord { @@ -50,54 +32,8 @@ pub struct CdcStateRecord { pub row_count: i64, } -impl CdcBackfillStateImpl { - pub fn init_epoch(&mut self, epoch: EpochPair) { - match self { - CdcBackfillStateImpl::SingleTable(state) => state.init_epoch(epoch), - CdcBackfillStateImpl::MultiTable(state) => state.init_epoch(epoch), - } - } - - /// Restore the state of the corresponding split - pub async fn restore_state(&mut self) -> StreamExecutorResult { - match self { - CdcBackfillStateImpl::SingleTable(state) => state.restore_state().await, - CdcBackfillStateImpl::MultiTable(state) => state.restore_state().await, - } - } - - /// Modify the state of the corresponding split (currently only supports single split) - pub async fn mutate_state( - &mut self, - current_pk_pos: Option, - last_cdc_offset: Option, - row_count: u64, - is_finished: bool, - ) -> StreamExecutorResult<()> { - let record = CdcStateRecord { - current_pk_pos, - last_cdc_offset, - row_count: row_count as _, - is_finished, - }; - match self { - CdcBackfillStateImpl::SingleTable(state) => state.mutate_state(&record).await, - CdcBackfillStateImpl::MultiTable(state) => state.mutate_state(&record).await, - } - } - - /// Persist the state to storage - pub async fn commit_state(&mut self, new_epoch: EpochPair) -> StreamExecutorResult<()> { - match self { - CdcBackfillStateImpl::SingleTable(state) => state.commit_state(new_epoch).await, - CdcBackfillStateImpl::MultiTable(state) => state.commit_state(new_epoch).await, - } - } -} - -pub const BACKFILL_STATE_KEY_SUFFIX: &str = "_backfill"; - -pub struct MultiBackfillState { +/// state schema: | `split_id` | `pk...` | `backfill_finished` | `row_count` | `cdc_offset` | +pub struct CdcBackfillState { /// Id of the backfilling table, will be the key of the state split_id: String, state_table: StateTable, @@ -105,7 +41,7 @@ pub struct MultiBackfillState { cached_state: Vec, } -impl MultiBackfillState { +impl CdcBackfillState { pub fn new(table_id: u32, state_table: StateTable, state_len: usize) -> Self { Self { split_id: table_id.to_string(), @@ -127,6 +63,7 @@ impl MultiBackfillState { .await? { Some(row) => { + tracing::info!("restored cdc backfill state: {:?}", row); self.cached_state = row.into_inner().into_vec(); let state = self.cached_state.as_slice(); let state_len = state.len(); @@ -158,18 +95,25 @@ impl MultiBackfillState { } } - pub async fn mutate_state(&mut self, record: &CdcStateRecord) -> StreamExecutorResult<()> { + /// Modify the state of the corresponding split (currently only supports single split) + pub async fn mutate_state( + &mut self, + current_pk_pos: Option, + last_cdc_offset: Option, + row_count: u64, + is_finished: bool, + ) -> StreamExecutorResult<()> { // schema: | `split_id` | `pk...` | `backfill_finished` | `row_count` | `cdc_offset` | let state = self.cached_state.as_mut_slice(); let split_id = Some(ScalarImpl::from(self.split_id.clone())); let state_len = state.len(); state[0] = split_id.clone(); - if let Some(current_pk_pos) = &record.current_pk_pos { + if let Some(current_pk_pos) = ¤t_pk_pos { state[1..=current_pk_pos.len()].clone_from_slice(current_pk_pos.as_inner()); } - state[state_len - 3] = Some(record.is_finished.into()); - state[state_len - 2] = Some(record.row_count.into()); - state[state_len - 1] = record.last_cdc_offset.clone().map(|cdc_offset| { + state[state_len - 3] = Some(is_finished.into()); + state[state_len - 2] = Some((row_count as i64).into()); + state[state_len - 1] = last_cdc_offset.clone().map(|cdc_offset| { let json = serde_json::to_value(cdc_offset).unwrap(); ScalarImpl::Jsonb(JsonbVal::from(json)) }); @@ -186,119 +130,8 @@ impl MultiBackfillState { Ok(()) } + /// Persist the state to storage pub async fn commit_state(&mut self, new_epoch: EpochPair) -> StreamExecutorResult<()> { self.state_table.commit(new_epoch).await } } - -/// The state manager for backfilling a single table -pub struct SingleBackfillState { - /// Stores the backfill done flag - source_state_handler: SourceStateTableHandler, - cdc_table_id: u32, - split_id: SplitId, - cdc_split: SplitImpl, -} - -impl SingleBackfillState { - pub fn new( - source_state_handler: SourceStateTableHandler, - cdc_table_id: u32, - split_id: SplitId, - cdc_split: SplitImpl, - ) -> Self { - Self { - source_state_handler, - cdc_table_id, - split_id, - cdc_split, - } - } - - pub fn init_epoch(&mut self, epoch: EpochPair) { - self.source_state_handler.init_epoch(epoch) - } - - pub async fn restore_state(&self) -> StreamExecutorResult { - let mut key = self.split_id.to_string(); - key.push_str(BACKFILL_STATE_KEY_SUFFIX); - let is_finished = match self.source_state_handler.get(key.into()).await? { - Some(row) => match row.datum_at(1) { - Some(ScalarRefImpl::Jsonb(jsonb_ref)) => jsonb_ref.as_bool()?, - _ => unreachable!("invalid backfill persistent state"), - }, - None => false, - }; - Ok(CdcStateRecord { - is_finished, - ..Default::default() - }) - } - - pub async fn mutate_state(&mut self, state_item: &CdcStateRecord) -> StreamExecutorResult<()> { - // skip if unfinished for single backfill - if !state_item.is_finished { - return Ok(()); - } - - // When single backfill is finished, we should persist two states: - // 1) a finish flag to denote the backfill has done - // 2) a consumed binlog offset to denote the last binlog offset - // which will be committed to the state store upon next barrier. - let mut key = self.split_id.to_string(); - key.push_str(BACKFILL_STATE_KEY_SUFFIX); - // write backfill finished flag - self.source_state_handler - .set( - key.into(), - JsonbVal::from(Value::Bool(state_item.is_finished)), - ) - .await?; - - if let SplitImpl::MysqlCdc(split) = &mut self.cdc_split - && let Some(state) = split.mysql_split.as_mut() - { - let start_offset = state_item.last_cdc_offset.as_ref().map(|cdc_offset| { - let source_offset = if let CdcOffset::MySql(o) = cdc_offset { - DebeziumSourceOffset { - file: Some(o.filename.clone()), - pos: Some(o.position), - ..Default::default() - } - } else { - DebeziumSourceOffset::default() - }; - - let mut server = "RW_CDC_".to_string(); - server.push_str(self.cdc_table_id.to_string().as_str()); - DebeziumOffset { - source_partition: hashmap! { - "server".to_string() => server - }, - source_offset, - // upstream heartbeat event would not emit to the cdc backfill executor, - // since we don't parse heartbeat event in the source parser. - is_heartbeat: false, - } - }); - - // persist the last binlog offset into split state - state.inner.start_offset = start_offset.map(|o| { - let value = serde_json::to_value(o).unwrap(); - value.to_string() - }); - state.inner.snapshot_done = true; - } - // write the last binlog offset that will be used upon recovery - self.source_state_handler - .set(self.split_id.clone(), self.cdc_split.encode_to_json()) - .await - } - - pub async fn commit_state(&mut self, new_epoch: EpochPair) -> StreamExecutorResult<()> { - self.source_state_handler - .state_store - .commit(new_epoch) - .await - } -} diff --git a/src/stream/src/executor/source/state_table_handler.rs b/src/stream/src/executor/source/state_table_handler.rs index d51d62ebfef06..82acd2a906f7f 100644 --- a/src/stream/src/executor/source/state_table_handler.rs +++ b/src/stream/src/executor/source/state_table_handler.rs @@ -37,7 +37,6 @@ use risingwave_storage::store::PrefetchOptions; use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; -use crate::executor::backfill::cdc::BACKFILL_STATE_KEY_SUFFIX; use crate::executor::error::StreamExecutorError; use crate::executor::StreamExecutorResult; @@ -217,44 +216,16 @@ impl SourceStateTableHandler { &mut self, stream_source_split: &SplitImpl, ) -> StreamExecutorResult> { - let split_id = stream_source_split.id(); - Ok(match self.get(split_id.clone()).await? { + Ok(match self.get(stream_source_split.id()).await? { None => None, Some(row) => match row.datum_at(1) { Some(ScalarRefImpl::Jsonb(jsonb_ref)) => { - let mut split_impl = SplitImpl::restore_from_json(jsonb_ref.to_owned_scalar())?; - if let SplitImpl::MysqlCdc(ref mut split) = split_impl - && let Some(mysql_split) = split.mysql_split.as_mut() - { - // if the snapshot_done is not set, we should check whether the backfill is finished - if !mysql_split.inner.snapshot_done { - mysql_split.inner.snapshot_done = - self.recover_cdc_snapshot_state(split_id).await?; - } - } - Some(split_impl) + Some(SplitImpl::restore_from_json(jsonb_ref.to_owned_scalar())?) } _ => unreachable!(), }, }) } - - async fn recover_cdc_snapshot_state( - &mut self, - split_id: SplitId, - ) -> StreamExecutorResult { - let mut key = split_id.to_string(); - key.push_str(BACKFILL_STATE_KEY_SUFFIX); - - let flag = match self.get(key.into()).await? { - Some(row) => match row.datum_at(1) { - Some(ScalarRefImpl::Jsonb(jsonb_ref)) => jsonb_ref.as_bool()?, - _ => unreachable!("invalid cdc backfill persistent state"), - }, - None => false, - }; - Ok(flag) - } } // align with schema defined in `LogicalSource::infer_internal_table_catalog`. The function is used diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index da5e789522eaf..8df56ee9d5d6a 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -12,9 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::catalog::{ColumnId, Schema, TableId}; -use risingwave_common::util::sort_util::OrderType; -use risingwave_connector::source::external::{CdcTableType, SchemaTableName}; +use risingwave_common::catalog::{ColumnId, TableId}; use risingwave_connector::source::{ConnectorProperties, SourceCtrlOpts}; use risingwave_pb::stream_plan::SourceNode; use risingwave_source::source_desc::SourceDescBuilder; @@ -22,11 +20,10 @@ use risingwave_storage::panic_store::PanicStateStore; use tokio::sync::mpsc::unbounded_channel; use super::*; -use crate::executor::external::ExternalStorageTable; use crate::executor::source::{FsListExecutor, StreamSourceCore}; use crate::executor::source_executor::SourceExecutor; use crate::executor::state_table_handler::SourceStateTableHandler; -use crate::executor::{CdcBackfillExecutor, FlowControlExecutor, FsSourceExecutor}; +use crate::executor::{FlowControlExecutor, FsSourceExecutor}; const FS_CONNECTORS: &[&str] = &["s3"]; pub struct SourceExecutorBuilder; @@ -129,7 +126,7 @@ impl ExecutorBuilder for SourceExecutorBuilder { ) .boxed() } else { - let source_exec = SourceExecutor::new( + SourceExecutor::new( params.actor_context.clone(), params.info.clone(), Some(stream_source_core), @@ -138,64 +135,8 @@ impl ExecutorBuilder for SourceExecutorBuilder { system_params, source_ctrl_opts.clone(), params.env.connector_params(), - ); - - let table_type = CdcTableType::from_properties(&source.properties); - if table_type.can_backfill() - && let Some(table_desc) = source_info.external_table.clone() - { - let table_schema = - Schema::new(table_desc.columns.iter().map(Into::into).collect()); - let upstream_table_name = - SchemaTableName::from_properties(&source.properties); - let table_pk_indices = table_desc - .pk - .iter() - .map(|k| k.column_index as usize) - .collect_vec(); - let table_pk_order_types = table_desc - .pk - .iter() - .map(|desc| OrderType::from_protobuf(desc.get_order_type().unwrap())) - .collect_vec(); - - let table_reader = table_type - .create_table_reader(source.properties.clone(), table_schema.clone()) - .await?; - let external_table = ExternalStorageTable::new( - TableId::new(source.source_id), - upstream_table_name, - table_reader, - table_schema.clone(), - table_pk_order_types, - table_pk_indices, - (0..table_schema.len()).collect_vec(), - ); - - // use the state table from source to store the backfill state (may refactor in future) - let source_state_handler = SourceStateTableHandler::from_table_catalog( - source.state_table.as_ref().unwrap(), - store.clone(), - ) - .await; - // use schema from table_desc - let cdc_backfill = CdcBackfillExecutor::new( - params.actor_context.clone(), - params.info.clone(), - external_table, - Box::new(source_exec), - (0..table_schema.len()).collect_vec(), - None, - params.executor_stats, - None, - Some(source_state_handler), - false, - source_ctrl_opts.chunk_size, - ); - cdc_backfill.boxed() - } else { - source_exec.boxed() - } + ) + .boxed() } }; let rate_limit = source.rate_limit.map(|x| x as _); diff --git a/src/stream/src/from_proto/stream_cdc_scan.rs b/src/stream/src/from_proto/stream_cdc_scan.rs index b96990cd44641..1736a38cecc1a 100644 --- a/src/stream/src/from_proto/stream_cdc_scan.rs +++ b/src/stream/src/from_proto/stream_cdc_scan.rs @@ -98,9 +98,7 @@ impl ExecutorBuilder for StreamCdcScanExecutorBuilder { output_indices, None, params.executor_stats, - Some(state_table), - None, - true, + state_table, params.env.config().developer.chunk_size, ) .boxed()) From 51c76aaa13ac4e81b61d37a07be1ac7afc6631a9 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Tue, 5 Dec 2023 13:13:28 +0800 Subject: [PATCH 003/215] feat(streaming): memory-size-based back-pressure in exchange (#13775) Signed-off-by: Runji Wang --- proto/task_service.proto | 2 +- src/common/src/config.rs | 22 ++++++++-------- src/config/example.toml | 4 +-- src/config/full-iceberg-bench.toml | 4 +-- src/stream/src/executor/exchange/input.rs | 2 +- src/stream/src/executor/exchange/permit.rs | 29 +++++++++++----------- src/stream/src/task/mod.rs | 8 +++--- 7 files changed, 35 insertions(+), 36 deletions(-) diff --git a/proto/task_service.proto b/proto/task_service.proto index ec72be346750f..d39a43ee5ce2a 100644 --- a/proto/task_service.proto +++ b/proto/task_service.proto @@ -82,7 +82,7 @@ message GetDataRequest { // The structure for permit-based back-pressure. message Permits { oneof value { - // The permits required for a chunk, i.e. the cardinality of the chunk. + // The permits required for a chunk, i.e. the total buffer size of the chunk. uint32 record = 1; // The permits required for a barrier, typically 1. uint32 barrier = 2; diff --git a/src/common/src/config.rs b/src/common/src/config.rs index a1e5d5fde7574..6028a3c485eb4 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -763,15 +763,13 @@ pub struct StreamingDeveloperConfig { #[serde(default = "default::developer::stream_chunk_size")] pub chunk_size: usize, - /// The initial permits that a channel holds, i.e., the maximum row count can be buffered in - /// the channel. - #[serde(default = "default::developer::stream_exchange_initial_permits")] - pub exchange_initial_permits: usize, + /// The maximum size of bytes can be buffered in the exchange channel. + #[serde(default = "default::developer::stream_exchange_max_bytes")] + pub exchange_max_bytes: usize, - /// The permits that are batched to add back, for reducing the backward `AddPermits` messages - /// in remote exchange. - #[serde(default = "default::developer::stream_exchange_batched_permits")] - pub exchange_batched_permits: usize, + /// The threshold of bytes that triggers a backward `AddPermits` message in the remote exchange. + #[serde(default = "default::developer::stream_exchange_ack_bytes")] + pub exchange_ack_bytes: usize, /// The maximum number of concurrent barriers in an exchange channel. #[serde(default = "default::developer::stream_exchange_concurrent_barriers")] @@ -1314,12 +1312,12 @@ pub mod default { 256 } - pub fn stream_exchange_initial_permits() -> usize { - 2048 + pub fn stream_exchange_max_bytes() -> usize { + 1 << 20 // 1MB } - pub fn stream_exchange_batched_permits() -> usize { - 256 + pub fn stream_exchange_ack_bytes() -> usize { + 32 << 10 // 32KB } pub fn stream_exchange_concurrent_barriers() -> usize { diff --git a/src/config/example.toml b/src/config/example.toml index fddd8f34bb79a..64be834425b54 100644 --- a/src/config/example.toml +++ b/src/config/example.toml @@ -82,8 +82,8 @@ stream_enable_executor_row_count = false stream_connector_message_buffer_size = 16 stream_unsafe_extreme_cache_size = 10 stream_chunk_size = 256 -stream_exchange_initial_permits = 2048 -stream_exchange_batched_permits = 256 +stream_exchange_max_bytes = 1048576 +stream_exchange_ack_bytes = 32768 stream_exchange_concurrent_barriers = 1 stream_exchange_concurrent_dispatchers = 0 stream_dml_channel_initial_permits = 32768 diff --git a/src/config/full-iceberg-bench.toml b/src/config/full-iceberg-bench.toml index 8c7cae6cd090a..45cc252209721 100644 --- a/src/config/full-iceberg-bench.toml +++ b/src/config/full-iceberg-bench.toml @@ -71,8 +71,8 @@ stream_enable_executor_row_count = false stream_connector_message_buffer_size = 16 stream_unsafe_extreme_cache_size = 10 stream_chunk_size = 256 -stream_exchange_initial_permits = 2048 -stream_exchange_batched_permits = 256 +stream_exchange_max_bytes = 1048576 +stream_exchange_ack_bytes = 32768 stream_exchange_concurrent_barriers = 1 stream_dml_channel_initial_permits = 32768 stream_hash_agg_max_dirty_groups_heap_size = 67108864 diff --git a/src/stream/src/executor/exchange/input.rs b/src/stream/src/executor/exchange/input.rs index 3804904b7c4f2..20644b26acd05 100644 --- a/src/stream/src/executor/exchange/input.rs +++ b/src/stream/src/executor/exchange/input.rs @@ -247,7 +247,7 @@ pub(crate) fn new_input( (upstream_actor_id, actor_id), (upstream_fragment_id, fragment_id), metrics, - context.config.developer.exchange_batched_permits, + context.config.developer.exchange_ack_bytes, ) .boxed_input() }; diff --git a/src/stream/src/executor/exchange/permit.rs b/src/stream/src/executor/exchange/permit.rs index 9c333ddf4b361..75a73b34fb02f 100644 --- a/src/stream/src/executor/exchange/permit.rs +++ b/src/stream/src/executor/exchange/permit.rs @@ -16,6 +16,7 @@ use std::sync::Arc; +use risingwave_common::estimate_size::EstimateSize; use risingwave_pb::task_service::permits; use tokio::sync::{mpsc, AcquireError, Semaphore, SemaphorePermit}; @@ -33,18 +34,18 @@ pub struct MessageWithPermits { /// Create a channel for the exchange service. pub fn channel( - initial_permits: usize, - batched_permits: usize, + max_bytes: usize, + ack_bytes: usize, concurrent_barriers: usize, ) -> (Sender, Receiver) { // Use an unbounded channel since we manage the permits manually. let (tx, rx) = mpsc::unbounded_channel(); - let records = Semaphore::new(initial_permits); + let records = Semaphore::new(max_bytes); let barriers = Semaphore::new(concurrent_barriers); let permits = Arc::new(Permits { records, barriers }); - let max_chunk_permits: usize = initial_permits - batched_permits; + let max_chunk_permits: usize = max_bytes - ack_bytes; ( Sender { @@ -58,15 +59,15 @@ pub fn channel( /// The configuration for tests. pub mod for_test { - pub const INITIAL_PERMITS: usize = (u32::MAX / 2) as _; - pub const BATCHED_PERMITS: usize = 1; + pub const MAX_BYTES: usize = (u32::MAX / 2) as _; + pub const ACK_BYTES: usize = 1; pub const CONCURRENT_BARRIERS: usize = (u32::MAX / 2) as _; } pub fn channel_for_test() -> (Sender, Receiver) { use for_test::*; - channel(INITIAL_PERMITS, BATCHED_PERMITS, CONCURRENT_BARRIERS) + channel(MAX_BYTES, ACK_BYTES, CONCURRENT_BARRIERS) } /// Semaphore-based permits to control the back-pressure. @@ -112,9 +113,8 @@ pub struct Sender { tx: mpsc::UnboundedSender, permits: Arc, - /// The maximum permits required by a chunk. If there're too many rows in a chunk, we only - /// acquire these permits. `BATCHED_PERMITS` is subtracted to avoid deadlock with - /// batching. + /// The maximum permits required by a chunk. If the chunk size is too large, we only + /// acquire these permits. `ack_bytes` is subtracted to avoid deadlock with batching. max_chunk_permits: usize, } @@ -126,11 +126,12 @@ impl Sender { // The semaphores should never be closed. let permits = match &message { Message::Chunk(c) => { - let card = c.cardinality().clamp(1, self.max_chunk_permits); - if card == self.max_chunk_permits { - tracing::warn!(cardinality = c.cardinality(), "large chunk in exchange") + let size = c.estimated_size(); + let permits = size.min(self.max_chunk_permits); + if permits == self.max_chunk_permits { + tracing::warn!(size, "large chunk in exchange") } - Some(permits::Value::Record(card as _)) + Some(permits::Value::Record(permits as _)) } Message::Barrier(_) => Some(permits::Value::Barrier(1)), Message::Watermark(_) => None, diff --git a/src/stream/src/task/mod.rs b/src/stream/src/task/mod.rs index 109d81e0b771d..9732b3a747e29 100644 --- a/src/stream/src/task/mod.rs +++ b/src/stream/src/task/mod.rs @@ -116,8 +116,8 @@ impl SharedContext { ))), config: StreamingConfig { developer: StreamingDeveloperConfig { - exchange_initial_permits: permit::for_test::INITIAL_PERMITS, - exchange_batched_permits: permit::for_test::BATCHED_PERMITS, + exchange_max_bytes: permit::for_test::MAX_BYTES, + exchange_ack_bytes: permit::for_test::ACK_BYTES, exchange_concurrent_barriers: permit::for_test::CONCURRENT_BARRIERS, ..Default::default() }, @@ -139,8 +139,8 @@ impl SharedContext { MutexGuard::map(self.channel_map.lock(), |map| { map.entry(ids).or_insert_with(|| { let (tx, rx) = permit::channel( - self.config.developer.exchange_initial_permits, - self.config.developer.exchange_batched_permits, + self.config.developer.exchange_max_bytes, + self.config.developer.exchange_ack_bytes, self.config.developer.exchange_concurrent_barriers, ); (Some(tx), Some(rx)) From 1c453f43463a0c1813c38d3c9435258da4efb2f3 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Tue, 5 Dec 2023 13:48:56 +0800 Subject: [PATCH 004/215] chore(integration_tests): remove invalid mysql type alias (#13721) --- integration_tests/mysql-cdc/mysql-datatypes.yml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/integration_tests/mysql-cdc/mysql-datatypes.yml b/integration_tests/mysql-cdc/mysql-datatypes.yml index 3ce26003c1347..8265a9f927120 100644 --- a/integration_tests/mysql-cdc/mysql-datatypes.yml +++ b/integration_tests/mysql-cdc/mysql-datatypes.yml @@ -86,8 +86,6 @@ datatypes: minimum: "'1001-01-01'" maximum: "'9999-12-31'" - name: time - aliases: - - time without time zone zero: "'00:00:00'" minimum: "'-838:59:59.000000'" maximum: "'838:59:59.000000'" @@ -97,8 +95,6 @@ datatypes: maximum: "'9999-12-31 23:59:59.499999'" rw_type: timestamp - name: timestamp - aliases: - - timestamp without time zone zero: "'1970-01-01 00:00:01'" minimum: "'1970-01-01 00:00:01'" maximum: "'2038-01-19 03:14:07'" From dfb4bc2782a39c73c3b34ec04d19a843bcc09872 Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Tue, 5 Dec 2023 16:44:23 +0800 Subject: [PATCH 005/215] fix(optimizer): do not push temporal filter predicate passing the temporal join (#13549) --- .../tests/testdata/input/temporal_filter.yaml | 9 ++- .../testdata/output/predicate_pushdown.yaml | 8 ++- .../testdata/output/temporal_filter.yaml | 16 +++++ .../src/optimizer/plan_node/generic/join.rs | 40 ++++++++--- .../src/optimizer/plan_node/logical_apply.rs | 4 +- .../src/optimizer/plan_node/logical_join.rs | 70 +++++-------------- 6 files changed, 77 insertions(+), 70 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/input/temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/input/temporal_filter.yaml index 9f07c136e5977..f42c07f77e519 100644 --- a/src/frontend/planner_test/tests/testdata/input/temporal_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/input/temporal_filter.yaml @@ -21,7 +21,7 @@ create table t1 (ts timestamp with time zone); select * from t1 where now() - interval '15 minutes' > ts; expected_outputs: - - stream_error + - stream_error - name: Temporal filter reorders now expressions correctly sql: | create table t1 (ts timestamp with time zone); @@ -71,3 +71,10 @@ select * from t1 right join t2 on a = b AND tb < now() - interval '1 hour' and tb >= now() - interval '2 hour'; expected_outputs: - stream_error +- name: Temporal filter after temporal join + sql: | + create table stream(id1 int, a1 int, b1 int, v1 timestamp with time zone) APPEND ONLY; + create table version(id2 int, a2 int, b2 int, primary key (id2)); + select id1, a1, id2, v1 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on id1 = id2 where v1 > now(); + expected_outputs: + - stream_plan diff --git a/src/frontend/planner_test/tests/testdata/output/predicate_pushdown.yaml b/src/frontend/planner_test/tests/testdata/output/predicate_pushdown.yaml index fde2853d81361..23723047a7cb9 100644 --- a/src/frontend/planner_test/tests/testdata/output/predicate_pushdown.yaml +++ b/src/frontend/planner_test/tests/testdata/output/predicate_pushdown.yaml @@ -280,7 +280,9 @@ ├─LogicalScan { table: t1, columns: [t1.v1] } └─LogicalProject { exprs: [t2.v2, t2.v3, ('2021-04-01 00:00:00+00:00':Timestamptz + t2.v3) as $expr1] } └─LogicalScan { table: t2, columns: [t2.v2, t2.v3] } - stream_error: Conditions containing now must be of the form `input_expr cmp now() [+- const_expr]` or `now() [+- const_expr] cmp input_expr`, where `input_expr` references a column and contains no `now()`. + stream_error: |- + Not supported: optimizer has tried to separate the temporal predicate(with now() expression) from the on condition, but it still reminded in on join's condition. Considering move it into WHERE clause? + HINT: please refer to https://www.risingwave.dev/docs/current/sql-pattern-temporal-filters/ for more information - name: now() in complex cmp expr pushed onto join ON clause results in dynamic filter sql: | create table t1(v1 timestamp with time zone); @@ -291,7 +293,9 @@ ├─LogicalScan { table: t1, columns: [t1.v1] } └─LogicalProject { exprs: [t2.v2, t2.v3, ('2021-04-01 00:00:00+00:00':Timestamptz + t2.v3) as $expr1] } └─LogicalScan { table: t2, columns: [t2.v2, t2.v3] } - stream_error: Conditions containing now must be of the form `input_expr cmp now() [+- const_expr]` or `now() [+- const_expr] cmp input_expr`, where `input_expr` references a column and contains no `now()`. + stream_error: |- + Not supported: optimizer has tried to separate the temporal predicate(with now() expression) from the on condition, but it still reminded in on join's condition. Considering move it into WHERE clause? + HINT: please refer to https://www.risingwave.dev/docs/current/sql-pattern-temporal-filters/ for more information - name: now() does not get pushed to scan, but others do sql: | create table t1(v1 timestamp with time zone, v2 int); diff --git a/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml index abfded30edb49..da395ae93622b 100644 --- a/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml @@ -203,3 +203,19 @@ stream_error: |- Not supported: optimizer has tried to separate the temporal predicate(with now() expression) from the on condition, but it still reminded in on join's condition. Considering move it into WHERE clause? HINT: please refer to https://www.risingwave.dev/docs/current/sql-pattern-temporal-filters/ for more information +- name: Temporal filter after temporal join + sql: | + create table stream(id1 int, a1 int, b1 int, v1 timestamp with time zone) APPEND ONLY; + create table version(id2 int, a2 int, b2 int, primary key (id2)); + select id1, a1, id2, v1 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on id1 = id2 where v1 > now(); + stream_plan: |- + StreamMaterialize { columns: [id1, a1, id2, v1, stream._row_id(hidden)], stream_key: [stream._row_id, id1], pk_columns: [stream._row_id, id1], pk_conflict: NoCheck, watermark_columns: [v1] } + └─StreamProject { exprs: [stream.id1, stream.a1, version.id2, stream.v1, stream._row_id], output_watermarks: [stream.v1] } + └─StreamDynamicFilter { predicate: (stream.v1 > now), output_watermarks: [stream.v1], output: [stream.id1, stream.a1, stream.v1, version.id2, stream._row_id], cleaned_by_watermark: true } + ├─StreamTemporalJoin { type: LeftOuter, predicate: stream.id1 = version.id2, output: [stream.id1, stream.a1, stream.v1, version.id2, stream._row_id] } + │ ├─StreamExchange { dist: HashShard(stream.id1) } + │ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.v1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + │ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } + │ └─StreamTableScan { table: version, columns: [version.id2], pk: [version.id2], dist: UpstreamHashShard(version.id2) } + └─StreamExchange { dist: Broadcast } + └─StreamNow { output: [now] } diff --git a/src/frontend/src/optimizer/plan_node/generic/join.rs b/src/frontend/src/optimizer/plan_node/generic/join.rs index e4630de5a3c36..c39ce1891078b 100644 --- a/src/frontend/src/optimizer/plan_node/generic/join.rs +++ b/src/frontend/src/optimizer/plan_node/generic/join.rs @@ -480,6 +480,7 @@ pub fn push_down_into_join( left_col_num: usize, right_col_num: usize, ty: JoinType, + push_temporal_predicate: bool, ) -> (Condition, Condition, Condition) { let (left, right) = push_down_to_inputs( predicate, @@ -487,19 +488,24 @@ pub fn push_down_into_join( right_col_num, can_push_left_from_filter(ty), can_push_right_from_filter(ty), + push_temporal_predicate, ); let on = if can_push_on_from_filter(ty) { let mut conjunctions = std::mem::take(&mut predicate.conjunctions); - // Do not push now on to the on, it will be pulled up into a filter instead. - let on = Condition { - conjunctions: conjunctions - .extract_if(|expr| expr.count_nows() == 0) - .collect(), - }; - predicate.conjunctions = conjunctions; - on + if push_temporal_predicate { + Condition { conjunctions } + } else { + // Do not push now on to the on, it will be pulled up into a filter instead. + let on = Condition { + conjunctions: conjunctions + .extract_if(|expr| expr.count_nows() == 0) + .collect(), + }; + predicate.conjunctions = conjunctions; + on + } } else { Condition::true_cond() }; @@ -516,6 +522,7 @@ pub fn push_down_join_condition( left_col_num: usize, right_col_num: usize, ty: JoinType, + push_temporal_predicate: bool, ) -> (Condition, Condition) { push_down_to_inputs( on_condition, @@ -523,6 +530,7 @@ pub fn push_down_join_condition( right_col_num, can_push_left_from_on(ty), can_push_right_from_on(ty), + push_temporal_predicate, ) } @@ -536,11 +544,21 @@ fn push_down_to_inputs( right_col_num: usize, push_left: bool, push_right: bool, + push_temporal_predicate: bool, ) -> (Condition, Condition) { - let conjunctions = std::mem::take(&mut predicate.conjunctions); + let mut conjunctions = std::mem::take(&mut predicate.conjunctions); + let (mut left, right, mut others) = if push_temporal_predicate { + Condition { conjunctions }.split(left_col_num, right_col_num) + } else { + let temporal_filter_cons = conjunctions + .extract_if(|e| e.count_nows() != 0) + .collect_vec(); + let (left, right, mut others) = + Condition { conjunctions }.split(left_col_num, right_col_num); - let (mut left, right, mut others) = - Condition { conjunctions }.split(left_col_num, right_col_num); + others.conjunctions.extend(temporal_filter_cons); + (left, right, others) + }; if !push_left { others.conjunctions.extend(left); diff --git a/src/frontend/src/optimizer/plan_node/logical_apply.rs b/src/frontend/src/optimizer/plan_node/logical_apply.rs index b4172892a7071..3d85e404d767d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_apply.rs +++ b/src/frontend/src/optimizer/plan_node/logical_apply.rs @@ -327,11 +327,11 @@ impl PredicatePushdown for LogicalApply { let join_type = self.join_type(); let (left_from_filter, right_from_filter, on) = - push_down_into_join(&mut predicate, left_col_num, right_col_num, join_type); + push_down_into_join(&mut predicate, left_col_num, right_col_num, join_type, true); let mut new_on = self.on.clone().and(on); let (left_from_on, right_from_on) = - push_down_join_condition(&mut new_on, left_col_num, right_col_num, join_type); + push_down_join_condition(&mut new_on, left_col_num, right_col_num, join_type, true); let left_predicate = left_from_filter.and(left_from_on); let right_predicate = right_from_filter.and(right_from_on); diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index dade239c7758f..33896b2b6d242 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -188,56 +188,6 @@ impl LogicalJoin { self.output_indices() == &(0..self.internal_column_num()).collect_vec() } - /// Try to split and pushdown `predicate` into a join's left/right child or the on clause. - /// Returns the pushed predicates. The pushed part will be removed from the original predicate. - /// - /// `InputRef`s in the right `Condition` are shifted by `-left_col_num`. - pub fn push_down( - predicate: &mut Condition, - left_col_num: usize, - right_col_num: usize, - push_left: bool, - push_right: bool, - push_on: bool, - ) -> (Condition, Condition, Condition) { - let conjunctions = std::mem::take(&mut predicate.conjunctions); - - let (mut left, right, mut others) = - Condition { conjunctions }.split(left_col_num, right_col_num); - - if !push_left { - others.conjunctions.extend(left); - left = Condition::true_cond(); - }; - - let right = if push_right { - let mut mapping = ColIndexMapping::with_shift_offset( - left_col_num + right_col_num, - -(left_col_num as isize), - ); - right.rewrite_expr(&mut mapping) - } else { - others.conjunctions.extend(right); - Condition::true_cond() - }; - - let on = if push_on { - // Do not push now on to the on, it will be pulled up into a filter instead. - Condition { - conjunctions: others - .conjunctions - .extract_if(|expr| expr.count_nows() == 0) - .collect(), - } - } else { - Condition::true_cond() - }; - - predicate.conjunctions = others.conjunctions; - - (left, right, on) - } - /// Try to simplify the outer join with the predicate on the top of the join /// /// now it is just a naive implementation for comparison expression, we can give a more general @@ -793,12 +743,24 @@ impl PredicatePushdown for LogicalJoin { let right_col_num = self.right().schema().len(); let join_type = LogicalJoin::simplify_outer(&predicate, left_col_num, self.join_type()); - let (left_from_filter, right_from_filter, on) = - push_down_into_join(&mut predicate, left_col_num, right_col_num, join_type); + let push_down_temporal_predicate = !self.should_be_temporal_join(); + + let (left_from_filter, right_from_filter, on) = push_down_into_join( + &mut predicate, + left_col_num, + right_col_num, + join_type, + push_down_temporal_predicate, + ); let mut new_on = self.on().clone().and(on); - let (left_from_on, right_from_on) = - push_down_join_condition(&mut new_on, left_col_num, right_col_num, join_type); + let (left_from_on, right_from_on) = push_down_join_condition( + &mut new_on, + left_col_num, + right_col_num, + join_type, + push_down_temporal_predicate, + ); let left_predicate = left_from_filter.and(left_from_on); let right_predicate = right_from_filter.and(right_from_on); From 7d1e9d20d5a1dae2c75968da934563b0712db6c2 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 5 Dec 2023 17:16:20 +0800 Subject: [PATCH 006/215] fix: apply `RW_WORKER_THREADS` for all runtimes (#13810) Signed-off-by: Bugen Zhao --- src/utils/runtime/src/lib.rs | 18 +++++++----------- 1 file changed, 7 insertions(+), 11 deletions(-) diff --git a/src/utils/runtime/src/lib.rs b/src/utils/runtime/src/lib.rs index 9986e09cc572c..343550246e243 100644 --- a/src/utils/runtime/src/lib.rs +++ b/src/utils/runtime/src/lib.rs @@ -35,10 +35,9 @@ use prof::*; /// /// Currently, the following env variables will be read: /// -/// * `RW_WORKER_THREADS`: number of tokio worker threads. If not set, it will use tokio's default -/// config (equivalent to CPU cores). Note: This will not effect the dedicated runtimes for each -/// service which are controlled by their own configurations, like streaming actors, compactions, -/// etc. +/// * `RW_WORKER_THREADS` (alias of `TOKIO_WORKER_THREADS`): number of tokio worker threads. If +/// not set, it will be decided by tokio. Note that this can still be overridden by per-module +/// runtime worker thread settings in the config file. /// * `RW_DEADLOCK_DETECTION`: whether to enable deadlock detection. If not set, will enable in /// debug mode, and disable in release mode. /// * `RW_PROFILE_PATH`: the path to generate flamegraph. If set, then profiling is automatically @@ -51,12 +50,9 @@ where risingwave_variables::init_server_start_time(); - let mut builder = tokio::runtime::Builder::new_multi_thread(); - - if let Ok(worker_threads) = std::env::var("RW_WORKER_THREADS") { - let worker_threads = worker_threads.parse().unwrap(); - tracing::info!("setting tokio worker threads to {}", worker_threads); - builder.worker_threads(worker_threads); + // `TOKIO` will be read by tokio. Duplicate `RW` for compatibility. + if let Some(worker_threads) = std::env::var_os("RW_WORKER_THREADS") { + std::env::set_var("TOKIO_WORKER_THREADS", worker_threads); } if let Ok(enable_deadlock_detection) = std::env::var("RW_DEADLOCK_DETECTION") { @@ -77,7 +73,7 @@ where spawn_prof_thread(profile_path); } - builder + tokio::runtime::Builder::new_multi_thread() .thread_name("risingwave-main") .enable_all() .build() From 4f90e37369fd49703d19954c5e533e05fbc7d331 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 5 Dec 2023 17:21:25 +0800 Subject: [PATCH 007/215] refactor(error): clean-up direct error formatting (part 1) (#13763) Signed-off-by: Bugen Zhao --- .cargo/config.toml | 2 +- .vscode/settings.json.example | 19 +++++- Cargo.lock | 4 ++ lints/src/format_error.rs | 7 +- src/batch/src/executor/aggregation/orderby.rs | 6 +- src/batch/src/executor/insert.rs | 6 +- src/batch/src/executor/mod.rs | 17 +++-- src/batch/src/rpc/service/task_service.rs | 16 +++-- src/batch/src/task/task_execution.rs | 9 +-- .../common_service/src/metrics_manager.rs | 5 +- .../common_service/src/observer_manager.rs | 9 +-- src/common/src/array/arrow.rs | 19 +++--- src/common/src/array/error.rs | 17 ++++- src/common/src/array/list_array.rs | 8 ++- src/common/src/array/proto_reader.rs | 67 ++++++++++--------- src/common/src/array/value_reader.rs | 14 ++-- src/common/src/catalog/mod.rs | 5 +- src/common/src/config.rs | 7 +- src/common/src/error.rs | 9 ++- src/common/src/metrics/guarded_metrics.rs | 5 +- src/common/src/telemetry/mod.rs | 3 +- src/common/src/telemetry/report.rs | 4 +- src/common/src/types/datetime.rs | 8 ++- src/common/src/types/mod.rs | 36 +++++----- src/common/src/util/addr.rs | 13 ++-- src/common/src/util/resource_util.rs | 11 ++- src/compute/Cargo.toml | 1 + src/compute/src/rpc/service/config_service.rs | 7 +- .../src/rpc/service/exchange_service.rs | 7 +- .../src/rpc/service/monitor_service.rs | 5 +- src/compute/src/rpc/service/stream_service.rs | 11 +-- src/compute/src/server.rs | 7 +- src/error/src/lib.rs | 3 + src/error/src/tonic.rs | 4 +- src/expr/core/Cargo.toml | 1 + src/expr/core/src/error.rs | 5 +- src/meta/src/barrier/mod.rs | 2 +- src/prost/Cargo.toml | 1 + src/prost/src/lib.rs | 3 +- src/sqlparser/src/lib.rs | 3 + src/sqlparser/test_runner/src/bin/apply.rs | 4 ++ src/sqlparser/test_runner/src/lib.rs | 6 +- src/storage/hummock_trace/src/error.rs | 2 +- src/stream/src/cache/managed_lru.rs | 15 +++-- .../src/common/log_store_impl/in_mem.rs | 6 +- src/stream/src/error.rs | 29 ++++++-- src/stream/src/executor/actor.rs | 5 +- src/stream/src/executor/error.rs | 2 +- src/stream/src/executor/exchange/input.rs | 14 ++-- src/stream/src/executor/merge.rs | 4 +- src/stream/src/executor/sink.rs | 3 +- src/stream/src/executor/sort_buffer.rs | 4 +- .../src/executor/source/fetch_executor.rs | 3 +- .../src/executor/source/list_executor.rs | 3 +- .../src/executor/source/source_executor.rs | 3 +- src/stream/src/executor/subtask.rs | 4 +- src/stream/src/from_proto/batch_query.rs | 5 +- src/stream/src/from_proto/stream_scan.rs | 5 +- src/stream/src/task/barrier_manager.rs | 15 +++-- .../src/task/barrier_manager/managed_state.rs | 20 +++--- src/stream/src/task/stream_manager.rs | 10 ++- src/tests/compaction_test/src/lib.rs | 3 + src/tests/e2e_extended_mode/src/lib.rs | 4 ++ src/tests/simulation/src/lib.rs | 3 + src/tests/sqlsmith/src/bin/main.rs | 4 ++ src/tests/sqlsmith/src/lib.rs | 3 + src/tests/state_cleaning_test/src/bin/main.rs | 4 ++ src/udf/src/external.rs | 8 ++- src/utils/pgwire/src/pg_protocol.rs | 8 +-- src/utils/pgwire/src/pg_server.rs | 11 +-- src/utils/runtime/Cargo.toml | 1 + src/utils/runtime/src/logger.rs | 21 ++++-- src/utils/runtime/src/prof.rs | 4 +- 73 files changed, 390 insertions(+), 232 deletions(-) diff --git a/.cargo/config.toml b/.cargo/config.toml index 2c41ca0110f64..fbe29b3e6b6af 100644 --- a/.cargo/config.toml +++ b/.cargo/config.toml @@ -34,7 +34,7 @@ rustflags = [ rustflags = [ "--cfg", "tokio_unstable", - # uncomment the following two lines to enable `TaskLocalAlloc` + # Uncomment the following two lines to enable `TaskLocalAlloc`. # "--cfg", # "enable_task_local_alloc", ] diff --git a/.vscode/settings.json.example b/.vscode/settings.json.example index aba691f0db508..380860bc378e2 100644 --- a/.vscode/settings.json.example +++ b/.vscode/settings.json.example @@ -9,16 +9,29 @@ "src/risedevtool/schemas/risedev.json": "risedev.yml", "src/risedevtool/schemas/risedev-profiles.user.json": "risedev-profiles.user.yml" }, - // // ============================================================================ + + // Optional: check with custom lints using `cargo dylint` + // + // "rust-analyzer.check.overrideCommand": [ + // "cargo", + // "dylint", + // "--all", + // "--workspace", + // "--", + // "--all-targets", + // "--message-format=json" + // ], + + // // =========================================================================== // // BEGIN SECTION: configurations for the development of deterministic testing - // // ============================================================================ + // // =========================================================================== // "rust-analyzer.cargo.extraEnv": { // "RUSTFLAGS": "--cfg tokio_unstable --cfg madsim" // }, // "rust-analyzer.runnableEnv": { // "RUSTFLAGS": "--cfg tokio_unstable --cfg madsim" // }, - // "rust-analyzer.checkOnSave.overrideCommand": [ + // "rust-analyzer.check.overrideCommand": [ // "cargo", // "check", // "-p", diff --git a/Cargo.lock b/Cargo.lock index cd582cfba90ce..0ee74a35b92c5 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7644,6 +7644,7 @@ dependencies = [ "serde", "serde_json", "tempfile", + "thiserror-ext", "tikv-jemalloc-ctl", "tokio-stream", "tower", @@ -7848,6 +7849,7 @@ dependencies = [ "smallvec", "static_assertions", "thiserror", + "thiserror-ext", "tracing", "workspace-hack", ] @@ -8279,6 +8281,7 @@ dependencies = [ "pbjson-build", "prost 0.12.1", "prost-helpers", + "risingwave_error", "serde", "strum", "thiserror", @@ -8371,6 +8374,7 @@ dependencies = [ "risingwave_common", "risingwave_variables", "rlimit", + "thiserror-ext", "time", "tracing", "tracing-opentelemetry", diff --git a/lints/src/format_error.rs b/lints/src/format_error.rs index 83ddd13edb12b..d0328dc82248b 100644 --- a/lints/src/format_error.rs +++ b/lints/src/format_error.rs @@ -17,7 +17,7 @@ use clippy_utils::macros::{ find_format_arg_expr, find_format_args, is_format_macro, macro_backtrace, }; use clippy_utils::ty::implements_trait; -use clippy_utils::{is_trait_method, match_function_call}; +use clippy_utils::{is_in_cfg_test, is_in_test_function, is_trait_method, match_function_call}; use rustc_ast::FormatArgsPiece; use rustc_hir::{Expr, ExprKind}; use rustc_lint::{LateContext, LateLintPass}; @@ -63,6 +63,11 @@ const TRACING_FIELD_DISPLAY: [&str; 3] = ["tracing_core", "field", "display"]; impl<'tcx> LateLintPass<'tcx> for FormatError { fn check_expr(&mut self, cx: &LateContext<'tcx>, expr: &'tcx Expr<'_>) { + // Ignore if in test code. + if is_in_cfg_test(cx.tcx, expr.hir_id) || is_in_test_function(cx.tcx, expr.hir_id) { + return; + } + // `%err`, `?err` in tracing events and spans. if let Some(args) = match_function_call(cx, expr, &TRACING_FIELD_DEBUG) .or_else(|| match_function_call(cx, expr, &TRACING_FIELD_DISPLAY)) diff --git a/src/batch/src/executor/aggregation/orderby.rs b/src/batch/src/executor/aggregation/orderby.rs index 7f5cde312818e..68d0e98f04845 100644 --- a/src/batch/src/executor/aggregation/orderby.rs +++ b/src/batch/src/executor/aggregation/orderby.rs @@ -14,7 +14,7 @@ use std::ops::Range; -use anyhow::anyhow; +use anyhow::Context; use risingwave_common::array::{Op, RowRef, StreamChunk}; use risingwave_common::estimate_size::EstimateSize; use risingwave_common::row::{OwnedRow, Row, RowExt}; @@ -25,7 +25,7 @@ use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_expr::aggregate::{ AggStateDyn, AggregateFunction, AggregateState, BoxedAggregateFunction, }; -use risingwave_expr::{ExprError, Result}; +use risingwave_expr::Result; /// `ProjectionOrderBy` is a wrapper of `AggregateFunction` that sorts rows by given columns and /// then projects columns. @@ -77,7 +77,7 @@ impl ProjectionOrderBy { fn push_row(&self, state: &mut State, row: RowRef<'_>) -> Result<()> { let key = memcmp_encoding::encode_row(row.project(&self.order_col_indices), &self.order_types) - .map_err(|e| ExprError::Internal(anyhow!("failed to encode row, error: {}", e)))?; + .context("failed to encode row")?; let projected_row = row.project(&self.arg_indices).to_owned_row(); state.unordered_values_estimated_heap_size += diff --git a/src/batch/src/executor/insert.rs b/src/batch/src/executor/insert.rs index 9c00aec04f9ad..eb2c67ddc4793 100644 --- a/src/batch/src/executor/insert.rs +++ b/src/batch/src/executor/insert.rs @@ -15,7 +15,7 @@ use std::iter::repeat; use std::sync::Arc; -use anyhow::anyhow; +use anyhow::Context; use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::array::{ @@ -231,8 +231,8 @@ impl BoxedExecutorBuilder for InsertExecutor { .map(|IndexAndExpr { index: i, expr: e }| { Ok(( i as usize, - build_from_prost(&e.ok_or_else(|| anyhow!("expression is None"))?) - .map_err(|e| anyhow!("failed to build expression: {}", e))?, + build_from_prost(&e.context("expression is None")?) + .context("failed to build expression")?, )) }) .collect::>>()?; diff --git a/src/batch/src/executor/mod.rs b/src/batch/src/executor/mod.rs index db5ff1f25f497..a4fc03ebf2e02 100644 --- a/src/batch/src/executor/mod.rs +++ b/src/batch/src/executor/mod.rs @@ -41,7 +41,7 @@ mod update; mod utils; mod values; -use anyhow::anyhow; +use anyhow::Context; use async_recursion::async_recursion; pub use delete::*; pub use expand::*; @@ -69,6 +69,7 @@ pub use sort_agg::*; pub use sort_over_window::SortOverWindowExecutor; pub use source::*; pub use table_function::*; +use thiserror_ext::AsReport; pub use top_n::TopNExecutor; pub use union::*; pub use update::*; @@ -183,12 +184,14 @@ impl<'a, C: Clone> ExecutorBuilder<'a, C> { impl<'a, C: BatchTaskContext> ExecutorBuilder<'a, C> { pub async fn build(&self) -> Result { - self.try_build().await.map_err(|e| { - let err_msg = format!("Failed to build executor: {e}"); - let plan_node_body = self.plan_node.get_node_body(); - error!("{err_msg}, plan node is: \n {plan_node_body:?}"); - anyhow!(err_msg).into() - }) + self.try_build() + .await + .inspect_err(|e| { + let plan_node = self.plan_node.get_node_body(); + error!(error = %e.as_report(), ?plan_node, "failed to build executor"); + }) + .context("failed to build executor") + .map_err(Into::into) } #[async_recursion] diff --git a/src/batch/src/rpc/service/task_service.rs b/src/batch/src/rpc/service/task_service.rs index 4395f948497ce..088483b31df78 100644 --- a/src/batch/src/rpc/service/task_service.rs +++ b/src/batch/src/rpc/service/task_service.rs @@ -22,6 +22,7 @@ use risingwave_pb::task_service::{ CancelTaskRequest, CancelTaskResponse, CreateTaskRequest, ExecuteRequest, GetDataResponse, TaskInfoResponse, }; +use thiserror_ext::AsReport; use tokio_stream::wrappers::ReceiverStream; use tonic::{Request, Response, Status}; @@ -93,7 +94,7 @@ impl TaskService for BatchServiceImpl { state_rx, ))), Err(e) => { - error!("failed to fire task {}", e); + error!(error = %e.as_report(), "failed to fire task"); Err(e.into()) } } @@ -146,8 +147,9 @@ impl TaskService for BatchServiceImpl { .await { error!( - "failed to build executors and trigger execution of Task {:?}: {}", - task_id, e + error = %e.as_report(), + ?task_id, + "failed to build executors and trigger execution" ); return Err(e.into()); } @@ -158,12 +160,12 @@ impl TaskService for BatchServiceImpl { // therefore we would only have one data output. output_id: 0, }; - let mut output = task.get_task_output(&pb_task_output_id).map_err(|e| { + let mut output = task.get_task_output(&pb_task_output_id).inspect_err(|e| { error!( - "failed to get task output of Task {:?} in local execution mode", - task_id + error = %e.as_report(), + ?task_id, + "failed to get task output in local execution mode", ); - e })?; let mut writer = GrpcExchangeWriter::new(tx.clone()); // Always spawn a task and do not block current function. diff --git a/src/batch/src/task/task_execution.rs b/src/batch/src/task/task_execution.rs index c9dc9f888a7e4..a0bc4c5bfb62c 100644 --- a/src/batch/src/task/task_execution.rs +++ b/src/batch/src/task/task_execution.rs @@ -34,6 +34,7 @@ use risingwave_pb::plan_common::CapturedExecutionContext; use risingwave_pb::task_service::task_info_response::TaskStatus; use risingwave_pb::task_service::{GetDataResponse, TaskInfoResponse}; use risingwave_pb::PbFieldNotFound; +use thiserror_ext::AsReport; use tokio::select; use tokio::task::JoinHandle; use tokio_metrics::TaskMonitor; @@ -644,7 +645,7 @@ impl BatchTaskExecution { ShutdownMsg::Init => { // There is no message received from shutdown channel, which means it caused // task failed. - error!("Batch task failed: {:?}", e); + error!(error = %e.as_report(), "Batch task failed"); error = Some(e); state = TaskStatus::Failed; break; @@ -671,7 +672,7 @@ impl BatchTaskExecution { let error = error.map(Arc::new); *self.failure.lock() = error.clone(); - let err_str = error.as_ref().map(|e| e.to_string()); + let err_str = error.as_ref().map(|e| e.to_report_string()); if let Err(e) = sender.close(error).await { match e { SenderError => { @@ -689,8 +690,8 @@ impl BatchTaskExecution { if let Err(e) = self.change_state_notify(state, state_tx, err_str).await { warn!( - "The status receiver in FE has closed so the status push is failed {:}", - e + error = %e.as_report(), + "The status receiver in FE has closed so the status push is failed", ); } diff --git a/src/common/common_service/src/metrics_manager.rs b/src/common/common_service/src/metrics_manager.rs index a2fbd12bda3f7..f44e4e08f2295 100644 --- a/src/common/common_service/src/metrics_manager.rs +++ b/src/common/common_service/src/metrics_manager.rs @@ -19,10 +19,11 @@ use std::sync::OnceLock; use hyper::{Body, Request, Response}; use prometheus::{Encoder, Registry, TextEncoder}; use risingwave_common::monitor::GLOBAL_METRICS_REGISTRY; +use thiserror_ext::AsReport; use tower::make::Shared; use tower::ServiceBuilder; use tower_http::add_extension::AddExtensionLayer; -use tracing::{info, warn}; +use tracing::{error, info, warn}; pub struct MetricsManager {} @@ -46,7 +47,7 @@ impl MetricsManager { let serve_future = hyper::Server::bind(&listen_socket_addr).serve(Shared::new(service)); if let Err(err) = serve_future.await { - eprintln!("server error: {}", err); + error!(error = %err.as_report(), "metrics service exited with error"); } }); listen_addr_clone diff --git a/src/common/common_service/src/observer_manager.rs b/src/common/common_service/src/observer_manager.rs index c799995e41adb..fa2303380eef5 100644 --- a/src/common/common_service/src/observer_manager.rs +++ b/src/common/common_service/src/observer_manager.rs @@ -18,6 +18,7 @@ use risingwave_pb::meta::subscribe_response::Info; use risingwave_pb::meta::{SubscribeResponse, SubscribeType}; use risingwave_rpc_client::error::RpcError; use risingwave_rpc_client::MetaClient; +use thiserror_ext::AsReport; use tokio::task::JoinHandle; use tonic::{Status, Streaming}; @@ -175,7 +176,7 @@ where /// call the `handle_initialization_notification` and `handle_notification` to update node data. pub async fn start(mut self) -> JoinHandle<()> { if let Err(err) = self.wait_init_notification().await { - tracing::warn!("Receives meta's notification err {:?}", err); + tracing::warn!(error = %err.as_report(), "Receives meta's notification err"); self.re_subscribe().await; } @@ -190,8 +191,8 @@ where } self.observer_states.handle_notification(resp.unwrap()); } - Err(e) => { - tracing::error!("Receives meta's notification err {:?}", e); + Err(err) => { + tracing::warn!(error = %err.as_report(), "Receives meta's notification err"); self.re_subscribe().await; } } @@ -211,7 +212,7 @@ where tracing::debug!("re-subscribe success"); self.rx = rx; if let Err(err) = self.wait_init_notification().await { - tracing::warn!("Receives meta's notification err {:?}", err); + tracing::warn!(error = %err.as_report(), "Receives meta's notification err"); continue; } else { break; diff --git a/src/common/src/array/arrow.rs b/src/common/src/array/arrow.rs index 7fc6d277bce9e..c2313e883aeab 100644 --- a/src/common/src/array/arrow.rs +++ b/src/common/src/array/arrow.rs @@ -45,15 +45,14 @@ pub fn to_record_batch_with_schema( if column.data_type() == field.data_type() { Ok(column) } else { - cast(&column, field.data_type()) - .map_err(|err| ArrayError::FromArrow(err.to_string())) + cast(&column, field.data_type()).map_err(ArrayError::from_arrow) } }) .try_collect::<_, _, ArrayError>()?; let opts = arrow_array::RecordBatchOptions::default().with_row_count(Some(chunk.capacity())); arrow_array::RecordBatch::try_new_with_options(schema, columns, &opts) - .map_err(|err| ArrayError::ToArrow(err.to_string())) + .map_err(ArrayError::to_arrow) } // Implement bi-directional `From` between `DataChunk` and `arrow_array::RecordBatch`. @@ -84,7 +83,7 @@ impl TryFrom<&DataChunk> for arrow_array::RecordBatch { let opts = arrow_array::RecordBatchOptions::default().with_row_count(Some(chunk.capacity())); arrow_array::RecordBatch::try_new_with_options(schema, columns, &opts) - .map_err(|err| ArrayError::ToArrow(err.to_string())) + .map_err(ArrayError::to_arrow) } } @@ -129,7 +128,7 @@ macro_rules! converts_generic { .unwrap() .try_into()?, )),)* - t => Err(ArrayError::FromArrow(format!("unsupported data type: {t:?}"))), + t => Err(ArrayError::from_arrow(format!("unsupported data type: {t:?}"))), } } } @@ -252,8 +251,8 @@ impl TryFrom<&DataType> for arrow_schema::DataType { datatype.as_ref().try_into()?, true, )))), - DataType::Serial => Err(ArrayError::ToArrow( - "Serial type is not supported to convert to arrow".to_string(), + DataType::Serial => Err(ArrayError::to_arrow( + "Serial type is not supported to convert to arrow", )), } } @@ -485,9 +484,9 @@ impl TryFrom<&arrow_array::LargeBinaryArray> for DecimalArray { .map(|o| { o.map(|s| { let s = std::str::from_utf8(s) - .map_err(|_| ArrayError::FromArrow(format!("invalid decimal: {s:?}")))?; + .map_err(|_| ArrayError::from_arrow(format!("invalid decimal: {s:?}")))?; s.parse() - .map_err(|_| ArrayError::FromArrow(format!("invalid decimal: {s:?}"))) + .map_err(|_| ArrayError::from_arrow(format!("invalid decimal: {s:?}"))) }) .transpose() }) @@ -521,7 +520,7 @@ impl TryFrom<&arrow_array::LargeStringArray> for JsonbArray { .map(|o| { o.map(|s| { s.parse() - .map_err(|_| ArrayError::FromArrow(format!("invalid json: {s}"))) + .map_err(|_| ArrayError::from_arrow(format!("invalid json: {s}"))) }) .transpose() }) diff --git a/src/common/src/array/error.rs b/src/common/src/array/error.rs index 0bcebc4120a0b..08b13673f275f 100644 --- a/src/common/src/array/error.rs +++ b/src/common/src/array/error.rs @@ -17,8 +17,11 @@ use std::convert::Infallible; pub use anyhow::anyhow; use risingwave_pb::PbFieldNotFound; use thiserror::Error; +use thiserror_ext::Construct; -#[derive(Error, Debug)] +use crate::error::BoxedError; + +#[derive(Error, Debug, Construct)] pub enum ArrayError { #[error("Pb decode error: {0}")] PbDecode(#[from] prost::DecodeError), @@ -34,10 +37,18 @@ pub enum ArrayError { ), #[error("Convert from arrow error: {0}")] - FromArrow(String), + FromArrow( + #[source] + #[backtrace] + BoxedError, + ), #[error("Convert to arrow error: {0}")] - ToArrow(String), + ToArrow( + #[source] + #[backtrace] + BoxedError, + ), } impl From for ArrayError { diff --git a/src/common/src/array/list_array.rs b/src/common/src/array/list_array.rs index 2ca83677d4d80..65b926e38e567 100644 --- a/src/common/src/array/list_array.rs +++ b/src/common/src/array/list_array.rs @@ -22,6 +22,7 @@ use bytes::{Buf, BufMut}; use itertools::Itertools; use risingwave_pb::data::{ListArrayData, PbArray, PbArrayType}; use serde::{Deserialize, Serializer}; +use thiserror_ext::AsReport; use super::{ Array, ArrayBuilder, ArrayBuilderImpl, ArrayImpl, ArrayResult, BoolArray, PrimitiveArray, @@ -766,7 +767,10 @@ impl ListValue { _ => {} } }; - Ok(Some(ScalarImpl::from_literal(&s, self.data_type)?)) + Ok(Some( + ScalarImpl::from_literal(&s, self.data_type) + .map_err(|e| e.to_report_string())?, + )) } /// Parse a double quoted non-array value. @@ -793,7 +797,7 @@ impl ListValue { _ => {} } }; - ScalarImpl::from_literal(&s, self.data_type) + ScalarImpl::from_literal(&s, self.data_type).map_err(|e| e.to_report_string()) } /// Unescape a string. diff --git a/src/common/src/array/proto_reader.rs b/src/common/src/array/proto_reader.rs index ae14300955b97..e13d17dea9398 100644 --- a/src/common/src/array/proto_reader.rs +++ b/src/common/src/array/proto_reader.rs @@ -14,7 +14,7 @@ use std::io::{Cursor, Read}; -use anyhow::anyhow; +use anyhow::Context; use byteorder::{BigEndian, ReadBytesExt}; use paste::paste; use risingwave_pb::data::{PbArray, PbArrayType}; @@ -112,32 +112,35 @@ fn read_bool_array(array: &PbArray, cardinality: usize) -> ArrayResult) -> ArrayResult { - match cursor.read_i32::() { - Ok(days) => Date::with_days(days).map_err(|e| anyhow!(e).into()), - Err(e) => bail!("Failed to read i32 from Date buffer: {}", e), - } + let days = cursor + .read_i32::() + .context("failed to read i32 from Date buffer")?; + + Ok(Date::with_days(days)?) } fn read_time(cursor: &mut Cursor<&[u8]>) -> ArrayResult